From d0ad25497bb1894fbf6bd2fd86fa0c8471941467 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 20 Feb 2020 12:49:24 +0800 Subject: [PATCH] executor: add more diagnosis rule to check some metrics exceed thresholds (#14843) --- executor/inspection_result.go | 167 ++++++++++++++++++++++++++++- executor/inspection_result_test.go | 128 +++++++++++++++++++--- infoschema/metric_table_def.go | 59 +++++----- 3 files changed, 311 insertions(+), 43 deletions(-) diff --git a/executor/inspection_result.go b/executor/inspection_result.go index c91ea9c3b820e..7ab5745a5ebda 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -368,7 +368,20 @@ func (criticalErrorInspection) inspect(ctx context.Context, sctx sessionctx.Cont return results } -func (thresholdCheckInspection) inspect(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { +func (c thresholdCheckInspection) inspect(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { + inspects := []func(context.Context, sessionctx.Context, inspectionFilter) []inspectionResult{ + c.inspectThreshold1, + c.inspectThreshold2, + } + var results []inspectionResult + for _, inspect := range inspects { + re := inspect(ctx, sctx, filter) + results = append(results, re...) + } + return results +} + +func (thresholdCheckInspection) inspectThreshold1(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { var rules = []struct { item string component string @@ -481,3 +494,155 @@ func (thresholdCheckInspection) inspect(ctx context.Context, sctx sessionctx.Con } return results } + +func (thresholdCheckInspection) inspectThreshold2(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { + var rules = []struct { + tp string + item string + tbl string + condition string + threshold float64 + isMin bool + }{ + { + tp: "tidb", + item: "tso-duration", + tbl: "pd_tso_wait_duration", + condition: "quantile=0.999", + threshold: 0.05, + }, + { + tp: "tidb", + item: "get-token-duration", + tbl: "tidb_get_token_duration", + condition: "quantile=0.999", + threshold: 0.001 * 10e5, // the unit is microsecond + }, + { + tp: "tidb", + item: "load-schema-duration", + tbl: "tidb_load_schema_duration", + condition: "quantile=0.99", + threshold: 1, + }, + { + tp: "tikv", + item: "scheduler-cmd-duration", + tbl: "tikv_scheduler_command_duration", + condition: "quantile=0.99", + threshold: 0.1, + }, + { + tp: "tikv", + item: "handle-snapshot-duration", + tbl: "tikv_handle_snapshot_duration", + threshold: 30, + }, + { + tp: "tikv", + item: "storage-write-duration", + tbl: "tikv_storage_async_request_duration", + condition: "type='write'", + threshold: 0.1, + }, + { + tp: "tikv", + item: "storage-snapshot-duration", + tbl: "tikv_storage_async_request_duration", + condition: "type='snapshot'", + threshold: 0.05, + }, + { + tp: "tikv", + item: "rocksdb-write-duration", + tbl: "tikv_engine_write_duration", + condition: "type='write_max'", + threshold: 0.1 * 10e5, // the unit is microsecond + }, + { + tp: "tikv", + item: "rocksdb-get-duration", + tbl: "tikv_engine_max_get_duration", + condition: "type='get_max'", + threshold: 0.05 * 10e5, // the unit is microsecond + }, + { + tp: "tikv", + item: "rocksdb-seek-duration", + tbl: "tikv_engine_max_seek_duration", + condition: "type='seek_max'", + threshold: 0.05 * 10e5, // the unit is microsecond + }, + { + tp: "tikv", + item: "scheduler-pending-cmd-count", + tbl: "tikv_scheduler_pending_commands", + threshold: 1000, + }, + { + tp: "tikv", + item: "index-block-cache-hit", + tbl: "tikv_block_index_cache_hit", + condition: "value > 0", + threshold: 0.95, + isMin: true, + }, + { + tp: "tikv", + item: "filter-block-cache-hit", + tbl: "tikv_block_filter_cache_hit", + condition: "value > 0", + threshold: 0.95, + isMin: true, + }, + { + tp: "tikv", + item: "data-block-cache-hit", + tbl: "tikv_block_data_cache_hit", + condition: "value > 0", + threshold: 0.80, + isMin: true, + }, + } + var results []inspectionResult + for _, rule := range rules { + if !filter.enable(rule.item) { + continue + } + var sql string + condition := rule.condition + if len(condition) > 0 { + condition = "where " + condition + } + if rule.isMin { + sql = fmt.Sprintf("select instance, min(value) as min_value from metric_schema.%s %s group by instance having min_value < %f;", rule.tbl, condition, rule.threshold) + } else { + sql = fmt.Sprintf("select instance, max(value) as max_value from metric_schema.%s %s group by instance having max_value > %f;", rule.tbl, condition, rule.threshold) + } + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithContext(ctx, sql) + if err != nil { + sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) + continue + } + for _, row := range rows { + actual := fmt.Sprintf("%.2f", row.GetFloat64(1)) + expected := "" + if rule.isMin { + expected = fmt.Sprintf("> %.2f", rule.threshold) + } else { + expected = fmt.Sprintf("< %.2f", rule.threshold) + } + result := inspectionResult{ + tp: rule.tp, + instance: row.GetString(0), + item: rule.item, + actual: actual, + expected: expected, + severity: "warning", + detail: sql, + } + results = append(results, result) + } + } + return results +} diff --git a/executor/inspection_result_test.go b/executor/inspection_result_test.go index ae31aa9205212..edee30bbd0adf 100644 --- a/executor/inspection_result_test.go +++ b/executor/inspection_result_test.go @@ -210,6 +210,19 @@ func (s *inspectionResultSuite) TestThresholdCheckInspection(c *C) { types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "sched_2", 10.0), types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "split_check", 10.0), }, + "pd_tso_wait_duration": {}, + "tidb_get_token_duration": {}, + "tidb_load_schema_duration": {}, + "tikv_scheduler_command_duration": {}, + "tikv_handle_snapshot_duration": {}, + "tikv_storage_async_request_duration": {}, + "tikv_engine_write_duration": {}, + "tikv_engine_max_get_duration": {}, + "tikv_engine_max_seek_duration": {}, + "tikv_scheduler_pending_commands": {}, + "tikv_block_index_cache_hit": {}, + "tikv_block_data_cache_hit": {}, + "tikv_block_filter_cache_hit": {}, } fpName := "github.com/pingcap/tidb/executor/mockMergeMockInspectionTables" @@ -246,22 +259,20 @@ func (s *inspectionResultSuite) TestThresholdCheckInspection(c *C) { )) // construct some mock normal data - mockData = map[string][][]types.Datum{ + mockData["tikv_thread_cpu"] = [][]types.Datum{ // columns: time, instance, name, value - "tikv_thread_cpu": { - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "cop_normal0", 1.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "cop_high1", 0.1), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "cop_low1", 1.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "grpc_1", 7.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "grpc_2", 0.21), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "raftstore_1", 1.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "apply_0", 1.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "store_read_norm1", 1.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "store_read_high2", 1.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "store_read_low0", 1.0), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "sched_2", 0.3), - types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "split_check", 0.5), - }, + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "cop_normal0", 1.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "cop_high1", 0.1), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "cop_low1", 1.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "grpc_1", 7.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "grpc_2", 0.21), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "raftstore_1", 1.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "apply_0", 1.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "store_read_norm1", 1.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "store_read_high2", 1.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "store_read_low0", 1.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "sched_2", 0.3), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "split_check", 0.5), } ctx = context.WithValue(context.Background(), "__mockInspectionTables", configurations) @@ -277,6 +288,93 @@ func (s *inspectionResultSuite) TestThresholdCheckInspection(c *C) { "grpc_cpu tikv tikv-0 7.21 < 7.20, config: server.grpc-concurrency=8 select instance, sum(value) as cpu from metric_schema.tikv_thread_cpu where name like 'grpc%' and time=now() group by instance")) } +func (s *inspectionResultSuite) TestThresholdCheckInspection2(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + // Mock for metric table data. + fpName := "github.com/pingcap/tidb/executor/mockMetricsTableData" + c.Assert(failpoint.Enable(fpName, "return"), IsNil) + defer func() { c.Assert(failpoint.Disable(fpName), IsNil) }() + + datetime := func(s string) types.Time { + t, err := types.ParseTime(tk.Se.GetSessionVars().StmtCtx, s, mysql.TypeDatetime, types.MaxFsp) + c.Assert(err, IsNil) + return t + } + + // construct some mock abnormal data + mockData := map[string][][]types.Datum{ + "pd_tso_wait_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "pd-0", 0.999, 0.06), + }, + "tidb_get_token_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tidb-0", 0.999, 0.02*10e5), + }, + "tidb_load_schema_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tidb-0", 0.99, 2.0), + }, + "tikv_scheduler_command_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "get", 0.99, 2.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "write", 0.99, 5.0), + }, + "tikv_handle_snapshot_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "gen", 0.999, 40.0), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "read", 0.999, 10.0), + }, + "tikv_storage_async_request_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "write", 0.999, 0.2), + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "snapshot", 0.999, 0.06), + }, + "tikv_engine_write_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "write_max", "kv", 0.2*10e5), + }, + "tikv_engine_max_get_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "get_max", "kv", 0.06*10e5), + }, + "tikv_engine_max_seek_duration": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "seek_max", "raft", 0.06*10e5), + }, + "tikv_scheduler_pending_commands": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", 1001.0), + }, + "tikv_block_index_cache_hit": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "kv", 0.94), + }, + "tikv_block_data_cache_hit": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "kv", 0.79), + }, + "tikv_block_filter_cache_hit": { + types.MakeDatums(datetime("2020-02-14 05:20:00"), "tikv-0", "kv", 0.93), + }, + "tikv_thread_cpu": {}, + } + + ctx := context.WithValue(context.Background(), "__mockMetricsTableData", mockData) + ctx = failpoint.WithHook(ctx, func(_ context.Context, fpname string) bool { + return fpname == fpName + }) + + rs, err := tk.Se.Execute(ctx, "select item, type, instance, value, reference, details from information_schema.inspection_result where rule='threshold-check' order by item") + c.Assert(err, IsNil) + result := tk.ResultSetToResultWithCtx(ctx, rs[0], Commentf("execute inspect SQL failed")) + c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0), Commentf("unexpected warnings: %+v", tk.Se.GetSessionVars().StmtCtx.GetWarnings())) + result.Check(testkit.Rows( + "data-block-cache-hit tikv tikv-0 0.79 > 0.80 select instance, min(value) as min_value from metric_schema.tikv_block_data_cache_hit where value > 0 group by instance having min_value < 0.800000;", + "filter-block-cache-hit tikv tikv-0 0.93 > 0.95 select instance, min(value) as min_value from metric_schema.tikv_block_filter_cache_hit where value > 0 group by instance having min_value < 0.950000;", + "get-token-duration tidb tidb-0 20000.00 < 1000.00 select instance, max(value) as max_value from metric_schema.tidb_get_token_duration where quantile=0.999 group by instance having max_value > 1000.000000;", + "handle-snapshot-duration tikv tikv-0 40.00 < 30.00 select instance, max(value) as max_value from metric_schema.tikv_handle_snapshot_duration group by instance having max_value > 30.000000;", + "index-block-cache-hit tikv tikv-0 0.94 > 0.95 select instance, min(value) as min_value from metric_schema.tikv_block_index_cache_hit where value > 0 group by instance having min_value < 0.950000;", + "load-schema-duration tidb tidb-0 2.00 < 1.00 select instance, max(value) as max_value from metric_schema.tidb_load_schema_duration where quantile=0.99 group by instance having max_value > 1.000000;", + "rocksdb-get-duration tikv tikv-0 60000.00 < 50000.00 select instance, max(value) as max_value from metric_schema.tikv_engine_max_get_duration where type='get_max' group by instance having max_value > 50000.000000;", + "rocksdb-seek-duration tikv tikv-0 60000.00 < 50000.00 select instance, max(value) as max_value from metric_schema.tikv_engine_max_seek_duration where type='seek_max' group by instance having max_value > 50000.000000;", + "rocksdb-write-duration tikv tikv-0 200000.00 < 100000.00 select instance, max(value) as max_value from metric_schema.tikv_engine_write_duration where type='write_max' group by instance having max_value > 100000.000000;", + "scheduler-cmd-duration tikv tikv-0 5.00 < 0.10 select instance, max(value) as max_value from metric_schema.tikv_scheduler_command_duration where quantile=0.99 group by instance having max_value > 0.100000;", + "scheduler-pending-cmd-count tikv tikv-0 1001.00 < 1000.00 select instance, max(value) as max_value from metric_schema.tikv_scheduler_pending_commands group by instance having max_value > 1000.000000;", + "storage-snapshot-duration tikv tikv-0 0.20 < 0.05 select instance, max(value) as max_value from metric_schema.tikv_storage_async_request_duration where type='snapshot' group by instance having max_value > 0.050000;", + "storage-write-duration tikv tikv-0 0.20 < 0.10 select instance, max(value) as max_value from metric_schema.tikv_storage_async_request_duration where type='write' group by instance having max_value > 0.100000;", + "tso-duration tidb pd-0 0.06 < 0.05 select instance, max(value) as max_value from metric_schema.pd_tso_wait_duration where quantile=0.999 group by instance having max_value > 0.050000;", + )) +} + func (s *inspectionResultSuite) TestCriticalErrorInspection(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) diff --git a/infoschema/metric_table_def.go b/infoschema/metric_table_def.go index f740ab059164a..dc9bcdecc1067 100644 --- a/infoschema/metric_table_def.go +++ b/infoschema/metric_table_def.go @@ -1471,12 +1471,12 @@ var MetricTableMap = map[string]MetricTableDef{ "tikv_engine_max_get_duration": { PromQL: `max(tikv_engine_get_micro_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, Labels: []string{"instance", "type", "db"}, - Comment: "The max time consumed when executing get operations", + Comment: "The max time consumed when executing get operations, the unit is microsecond", }, "tikv_engine_avg_get_duration": { PromQL: `avg(tikv_engine_get_micro_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, Labels: []string{"instance", "type", "db"}, - Comment: "The average time consumed when executing get operations", + Comment: "The average time consumed when executing get operations, the unit is microsecond", }, "tikv_engine_seek_operations": { PromQL: `sum(rate(tikv_engine_locate{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (db,type,instance)`, @@ -1486,12 +1486,12 @@ var MetricTableMap = map[string]MetricTableDef{ "tikv_engine_max_seek_duration": { PromQL: `max(tikv_engine_seek_micro_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, Labels: []string{"instance", "type", "db"}, - Comment: "The time consumed when executing seek operation", + Comment: "The time consumed when executing seek operation, the unit is microsecond", }, "tikv_engine_avg_seek_duration": { PromQL: `avg(tikv_engine_seek_micro_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, Labels: []string{"instance", "type", "db"}, - Comment: "The time consumed when executing seek operation", + Comment: "The time consumed when executing seek operation, the unit is microsecond", }, "tikv_engine_write_operations": { PromQL: `sum(rate(tikv_engine_write_served{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (db,type,instance)`, @@ -1501,7 +1501,7 @@ var MetricTableMap = map[string]MetricTableDef{ "tikv_engine_write_duration": { PromQL: `max(tikv_engine_write_micro_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, Labels: []string{"instance", "type", "db"}, - Comment: "The time consumed when executing write operation", + Comment: "The time consumed when executing write operation, the unit is microsecond", }, "tikv_engine_wal_sync_operations": { PromQL: `sum(rate(tikv_engine_wal_file_synced{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (db,instance)`, @@ -1512,12 +1512,12 @@ var MetricTableMap = map[string]MetricTableDef{ "tikv_wal_sync_max_duration": { PromQL: `max(tikv_engine_wal_file_sync_micro_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, Labels: []string{"instance", "type", "db"}, - Comment: "The max time consumed when executing WAL sync operation", + Comment: "The max time consumed when executing WAL sync operation, the unit is microsecond", }, "tikv_wal_sync_duration": { PromQL: `avg(tikv_engine_wal_file_sync_micro_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, Labels: []string{"instance", "type"}, - Comment: "The time consumed when executing WAL sync operation", + Comment: "The time consumed when executing WAL sync operation, the unit is microsecond", }, "tikv_compaction_operations": { PromQL: `sum(rate(tikv_engine_event_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance,db)`, @@ -1570,28 +1570,28 @@ var MetricTableMap = map[string]MetricTableDef{ Comment: "The block cache size. Broken down by column family if shared block cache is disabled.", }, "tikv_block_all_cache_hit": { - PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_hit"}[$RANGE_DURATION])) by (db) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_hit"}[$RANGE_DURATION])) by (db) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_miss"}[$RANGE_DURATION])) by (db))`, - Labels: []string{"db"}, + PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_hit"}[$RANGE_DURATION])) by (db,instance) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_hit"}[$RANGE_DURATION])) by (db,instance) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_miss"}[$RANGE_DURATION])) by (db,instance))`, + Labels: []string{"instance", "db"}, Comment: "The hit rate of all block cache", }, "tikv_block_data_cache_hit": { - PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_data_hit"}[$RANGE_DURATION])) by (db) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_data_hit"}[$RANGE_DURATION])) by (db) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_data_miss"}[$RANGE_DURATION])) by (db))`, - Labels: []string{"db"}, + PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_data_hit"}[$RANGE_DURATION])) by (db,instance) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_data_hit"}[$RANGE_DURATION])) by (db,instance) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_data_miss"}[$RANGE_DURATION])) by (db,instance))`, + Labels: []string{"instance", "db"}, Comment: "The hit rate of data block cache", }, "tikv_block_filter_cache_hit": { - PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_filter_hit"}[$RANGE_DURATION])) by (db) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_filter_hit"}[$RANGE_DURATION])) by (db) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_filter_miss"}[$RANGE_DURATION])) by (db))`, - Labels: []string{"db"}, + PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_filter_hit"}[$RANGE_DURATION])) by (db,instance) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_filter_hit"}[$RANGE_DURATION])) by (db,instance) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_filter_miss"}[$RANGE_DURATION])) by (db,instance))`, + Labels: []string{"instance", "db"}, Comment: "The hit rate of data block cache", }, "tikv_block_index_cache_hit": { - PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_index_hit"}[$RANGE_DURATION])) by (db) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_index_hit"}[$RANGE_DURATION])) by (db) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_index_miss"}[$RANGE_DURATION])) by (db))`, - Labels: []string{"db"}, + PromQL: `sum(rate(tikv_engine_cache_efficiency{type="block_cache_index_hit"}[$RANGE_DURATION])) by (db,instance) / (sum(rate(tikv_engine_cache_efficiency{type="block_cache_index_hit"}[$RANGE_DURATION])) by (db,instance) + sum(rate(tikv_engine_cache_efficiency{type="block_cache_index_miss"}[$RANGE_DURATION])) by (db,instance))`, + Labels: []string{"instance", "db"}, Comment: "The hit rate of data block cache", }, "tikv_block_bloom_prefix_cache_hit": { - PromQL: `sum(rate(tikv_engine_bloom_efficiency{type="bloom_prefix_useful"}[$RANGE_DURATION])) by (db) / sum(rate(tikv_engine_bloom_efficiency{type="bloom_prefix_checked"}[$RANGE_DURATION])) by (db)`, - Labels: []string{"db"}, + PromQL: `sum(rate(tikv_engine_bloom_efficiency{type="bloom_prefix_useful"}[$RANGE_DURATION])) by (db,instance) / sum(rate(tikv_engine_bloom_efficiency{type="bloom_prefix_checked"}[$RANGE_DURATION])) by (db,instance)`, + Labels: []string{"instance", "db"}, Comment: "The hit rate of data block cache", }, "tikv_corrrput_keys_flow": { @@ -1700,36 +1700,41 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "type", "db"}, }, "tikv_engine_blob_seek_duration": { - PromQL: `avg(tikv_engine_blob_seek_micros_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, - Labels: []string{"instance", "type", "db"}, + PromQL: `avg(tikv_engine_blob_seek_micros_seconds{$LABEL_CONDITIONS}) by (db,type,instance)`, + Labels: []string{"instance", "type", "db"}, + Comment: "the unit is microsecond", }, "tikv_engine_blob_seek_operations": { PromQL: `sum(rate(tikv_engine_blob_locate{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (db,type,instance)`, Labels: []string{"instance", "type", "db"}, }, "tikv_engine_blob_get_duration": { - PromQL: `avg(tikv_engine_blob_get_micros_seconds{$LABEL_CONDITIONS}) by (type,db,instance)`, - Labels: []string{"instance", "type", "db"}, + PromQL: `avg(tikv_engine_blob_get_micros_seconds{$LABEL_CONDITIONS}) by (type,db,instance)`, + Labels: []string{"instance", "type", "db"}, + Comment: "the unit is microsecond", }, "tikv_engine_blob_bytes_flow": { PromQL: `sum(rate(tikv_engine_blob_flow_bytes{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance,db)`, Labels: []string{"instance", "type", "db"}, }, "tikv_engine_blob_file_read_duration": { - PromQL: `avg(tikv_engine_blob_file_read_micros_seconds{$LABEL_CONDITIONS}) by (type,instance,db)`, - Labels: []string{"instance", "type", "db"}, + PromQL: `avg(tikv_engine_blob_file_read_micros_seconds{$LABEL_CONDITIONS}) by (type,instance,db)`, + Labels: []string{"instance", "type", "db"}, + Comment: "the unit is microsecond", }, "tikv_engine_blob_file_write_duration": { - PromQL: `avg(tikv_engine_blob_file_write_micros_seconds{$LABEL_CONDITIONS}) by (type,instance,db)`, - Labels: []string{"instance", "type", "db"}, + PromQL: `avg(tikv_engine_blob_file_write_micros_seconds{$LABEL_CONDITIONS}) by (type,instance,db)`, + Labels: []string{"instance", "type", "db"}, + Comment: "the unit is microsecond", }, "tikv_engine_blob_file_sync_operations": { PromQL: `sum(rate(tikv_engine_blob_file_synced{$LABEL_CONDITIONS}[$RANGE_DURATION]))`, Labels: []string{"instance"}, }, "tikv_engine_blob_file_sync_duration": { - PromQL: `avg(tikv_engine_blob_file_sync_micros_seconds{$LABEL_CONDITIONS}) by (instance,type,db)`, - Labels: []string{"instance", "type", "db"}, + PromQL: `avg(tikv_engine_blob_file_sync_micros_seconds{$LABEL_CONDITIONS}) by (instance,type,db)`, + Labels: []string{"instance", "type", "db"}, + Comment: "the unit is microsecond", }, "tikv_engine_blob_file_count": { PromQL: `avg(tikv_engine_titandb_num_obsolete_blob_file{$LABEL_CONDITIONS}) by (instance,db)`,