Skip to content

Commit

Permalink
execdetails: remove useless lmSkipRows (#58076)
Browse files Browse the repository at this point in the history
  • Loading branch information
Lloyd-Pottiger authored Dec 25, 2024
1 parent e1c78f2 commit 1fef963
Show file tree
Hide file tree
Showing 2 changed files with 4 additions and 10 deletions.
6 changes: 0 additions & 6 deletions pkg/util/execdetails/execdetails.go
Original file line number Diff line number Diff line change
Expand Up @@ -783,7 +783,6 @@ func (e *basicCopRuntimeStats) mergeExecSummary(summary *tipb.ExecutorExecutionS
mvccInputRows: tiflashScanContext.GetMvccInputRows(),
mvccInputBytes: tiflashScanContext.GetMvccInputBytes(),
mvccOutputRows: tiflashScanContext.GetMvccOutputRows(),
lmSkipRows: tiflashScanContext.GetLmSkipRows(),
totalBuildBitmapMs: tiflashScanContext.GetTotalBuildBitmapMs(),
totalBuildInputStreamMs: tiflashScanContext.GetTotalBuildInputstreamMs(),
staleReadRegions: tiflashScanContext.GetStaleReadRegions(),
Expand Down Expand Up @@ -1025,7 +1024,6 @@ type TiFlashScanContext struct {
mvccInputRows uint64
mvccInputBytes uint64
mvccOutputRows uint64
lmSkipRows uint64
totalBuildBitmapMs uint64
totalBuildInputStreamMs uint64
staleReadRegions uint64
Expand Down Expand Up @@ -1070,7 +1068,6 @@ func (context *TiFlashScanContext) Clone() TiFlashScanContext {
mvccInputRows: context.mvccInputRows,
mvccInputBytes: context.mvccInputBytes,
mvccOutputRows: context.mvccOutputRows,
lmSkipRows: context.lmSkipRows,
totalBuildBitmapMs: context.totalBuildBitmapMs,
totalBuildInputStreamMs: context.totalBuildInputStreamMs,
staleReadRegions: context.staleReadRegions,
Expand Down Expand Up @@ -1140,7 +1137,6 @@ func (context *TiFlashScanContext) String() string {
"mvcc_input_rows:%d, "+
"mvcc_input_bytes:%d, "+
"mvcc_output_rows:%d, "+
"lm_skip_rows:%d, "+
"local_regions:%d, "+
"remote_regions:%d, "+
"tot_learner_read:%dms, "+
Expand Down Expand Up @@ -1169,7 +1165,6 @@ func (context *TiFlashScanContext) String() string {
context.mvccInputRows,
context.mvccInputBytes,
context.mvccOutputRows,
context.lmSkipRows,
context.localRegions,
context.remoteRegions,
context.totalLearnerReadMs,
Expand Down Expand Up @@ -1221,7 +1216,6 @@ func (context *TiFlashScanContext) Merge(other TiFlashScanContext) {
context.mvccInputRows += other.mvccInputRows
context.mvccInputBytes += other.mvccInputBytes
context.mvccOutputRows += other.mvccOutputRows
context.lmSkipRows += other.lmSkipRows
context.totalBuildBitmapMs += other.totalBuildBitmapMs
context.totalBuildInputStreamMs += other.totalBuildInputStreamMs
context.staleReadRegions += other.staleReadRegions
Expand Down
8 changes: 4 additions & 4 deletions pkg/util/execdetails/execdetails_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,15 +277,15 @@ func TestCopRuntimeStatsForTiFlash(t *testing.T) {
require.True(t, stats.ExistsCopStats(tableScanID))

cop := stats.GetOrCreateCopStats(tableScanID, kv.TiFlash)
require.Equal(t, "tiflash_task:{proc max:2ns, min:1ns, avg: 1ns, p80:2ns, p95:2ns, iters:3, tasks:2, threads:2}, tiflash_wait: {minTSO_wait: 20ms, pipeline_breaker_wait: 5ms, pipeline_queue_wait: 10ms}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, lm_skip_rows:0, local_regions:10, remote_regions:4, tot_learner_read:1ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:40ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:8192, data_skipped_rows:0, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:15ms, tot_read:202ms, disagg_cache_hit_bytes: 100, disagg_cache_miss_bytes: 50}}", cop.String())
require.Equal(t, "tiflash_task:{proc max:2ns, min:1ns, avg: 1ns, p80:2ns, p95:2ns, iters:3, tasks:2, threads:2}, tiflash_wait: {minTSO_wait: 20ms, pipeline_breaker_wait: 5ms, pipeline_queue_wait: 10ms}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, local_regions:10, remote_regions:4, tot_learner_read:1ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:40ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:8192, data_skipped_rows:0, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:15ms, tot_read:202ms, disagg_cache_hit_bytes: 100, disagg_cache_miss_bytes: 50}}", cop.String())

copStats := cop.stats["8.8.8.8"]
require.NotNil(t, copStats)

copStats.SetRowNum(10)
copStats.Record(time.Second, 10)
require.Equal(t, "time:1s, loops:2, threads:1, tiflash_wait: {minTSO_wait: 30ms, pipeline_breaker_wait: 10ms, pipeline_queue_wait: 20ms}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, lm_skip_rows:0, local_regions:10, remote_regions:4, tot_learner_read:1ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:40ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:8192, data_skipped_rows:0, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:15ms, tot_read:200ms, disagg_cache_hit_bytes: 100, disagg_cache_miss_bytes: 50}}", copStats.String())
expected := "tiflash_task:{proc max:4ns, min:3ns, avg: 3ns, p80:4ns, p95:4ns, iters:7, tasks:2, threads:2}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, lm_skip_rows:0, local_regions:6, remote_regions:2, tot_learner_read:0ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:50ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:20192, data_skipped_rows:86000, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:100ms, tot_read:3000ms, disagg_cache_hit_bytes: 20, disagg_cache_miss_bytes: 0}}"
require.Equal(t, "time:1s, loops:2, threads:1, tiflash_wait: {minTSO_wait: 30ms, pipeline_breaker_wait: 10ms, pipeline_queue_wait: 20ms}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, local_regions:10, remote_regions:4, tot_learner_read:1ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:40ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:8192, data_skipped_rows:0, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:15ms, tot_read:200ms, disagg_cache_hit_bytes: 100, disagg_cache_miss_bytes: 50}}", copStats.String())
expected := "tiflash_task:{proc max:4ns, min:3ns, avg: 3ns, p80:4ns, p95:4ns, iters:7, tasks:2, threads:2}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, local_regions:6, remote_regions:2, tot_learner_read:0ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:50ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:20192, data_skipped_rows:86000, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:100ms, tot_read:3000ms, disagg_cache_hit_bytes: 20, disagg_cache_miss_bytes: 0}}"
require.Equal(t, expected, stats.GetOrCreateCopStats(aggID, kv.TiFlash).String())

rootStats := stats.GetRootStats(tableReaderID)
Expand All @@ -302,7 +302,7 @@ func TestVectorSearchStats(t *testing.T) {
execSummary.DetailInfo.(*tipb.ExecutorExecutionSummary_TiflashScanContext).TiflashScanContext.TotalVectorIdxLoadFromS3 = &v
stats.RecordOneCopTask(1, kv.TiFlash, "8.8.8.8", execSummary)
s := stats.GetOrCreateCopStats(1, kv.TiFlash)
require.Equal(t, "tiflash_task:{time:0s, loops:0, threads:0}, vector_idx:{load:{total:0ms,from_s3:1,from_disk:0,from_cache:0},search:{total:0ms,visited_nodes:0,discarded_nodes:0},read:{vec_total:0ms,others_total:0ms}}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, lm_skip_rows:0, local_regions:0, remote_regions:0, tot_learner_read:0ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:0ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:0, data_skipped_rows:0, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:0ms, tot_read:0ms}}", s.String())
require.Equal(t, "tiflash_task:{time:0s, loops:0, threads:0}, vector_idx:{load:{total:0ms,from_s3:1,from_disk:0,from_cache:0},search:{total:0ms,visited_nodes:0,discarded_nodes:0},read:{vec_total:0ms,others_total:0ms}}, tiflash_scan:{mvcc_input_rows:0, mvcc_input_bytes:0, mvcc_output_rows:0, local_regions:0, remote_regions:0, tot_learner_read:0ms, region_balance:none, delta_rows:0, delta_bytes:0, segments:0, stale_read_regions:0, tot_build_snapshot:0ms, tot_build_bitmap:0ms, tot_build_inputstream:0ms, min_local_stream:0ms, max_local_stream:0ms, dtfile:{data_scanned_rows:0, data_skipped_rows:0, mvcc_scanned_rows:0, mvcc_skipped_rows:0, lm_filter_scanned_rows:0, lm_filter_skipped_rows:0, tot_rs_index_check:0ms, tot_read:0ms}}", s.String())
}

func TestRuntimeStatsWithCommit(t *testing.T) {
Expand Down

0 comments on commit 1fef963

Please sign in to comment.