Skip to content

Commit

Permalink
sqlstats: generate PCT_RUNTIME sort when requested
Browse files Browse the repository at this point in the history
Previously, when a request was made to the combined stats endpoint on
CRDB with a `PCT_RUNTIME` sort, we'd default to a `svcLatency` sort
instead because of a missing switch case.

This change fills that gap and adds some datadriven tests to inspect
the resulting `WHERE` clauses.

Fixes: #123841
Epic: CRDB-37544

Release note (ui change): Viewing SQL Activity sorted by `% of
Runtime` now correctly sorts entries by the runtime amount.
  • Loading branch information
dhartunian committed May 9, 2024
1 parent 0ad22c6 commit a8636af
Show file tree
Hide file tree
Showing 4 changed files with 245 additions and 0 deletions.
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -419,6 +419,7 @@ go_test(
"api_v2_sql_test.go",
"api_v2_test.go",
"bench_test.go",
"combined_statement_stats_test.go",
"config_test.go",
"connectivity_test.go",
"critical_nodes_test.go",
Expand Down
2 changes: 2 additions & 0 deletions pkg/server/combined_statement_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -546,6 +546,8 @@ func getStmtColumnFromSortOption(sort serverpb.StatsSortOptions) string {
return sortRetriesDesc
case serverpb.StatsSortOptions_LAST_EXEC:
return sortLastExecDesc
case serverpb.StatsSortOptions_PCT_RUNTIME:
return sortPCTRuntimeDesc
default:
return sortSvcLatDesc
}
Expand Down
74 changes: 74 additions & 0 deletions pkg/server/combined_statement_stats_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
// Copyright 2024 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package server

import (
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/datadriven"
)

func TestGetCombinedStatementsQueryClausesAndArgs(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

settings := cluster.MakeClusterSettings()
testingKnobs := &sqlstats.TestingKnobs{}

datadriven.RunTest(t, datapathutils.TestDataPath(t, "combined_statement_stats"), func(t *testing.T, d *datadriven.TestData) string {
switch d.Cmd {
case "query":
var limit int64
var start int64
var end int64
var sortString string
if d.HasArg("sort") {
d.ScanArgs(t, "sort", &sortString)
}
if d.HasArg("limit") {
d.ScanArgs(t, "limit", &limit)
}
if d.HasArg("start") {
d.ScanArgs(t, "start", &start)
}
if d.HasArg("end") {
d.ScanArgs(t, "end", &end)
}

gotWhereClause, gotOrderAndLimitClause, gotArgs := getCombinedStatementsQueryClausesAndArgs(
&serverpb.CombinedStatementsStatsRequest{
Start: start,
End: end,
FetchMode: &serverpb.CombinedStatementsStatsRequest_FetchMode{
StatsType: serverpb.CombinedStatementsStatsRequest_StmtStatsOnly,
Sort: serverpb.StatsSortOptions(serverpb.StatsSortOptions_value[sortString]),
},
Limit: limit,
},
testingKnobs,
false,
settings,
)

return fmt.Sprintf("--WHERE--\n%s\n--ORDER AND LIMIT--\n%s\n--ARGS--\n%v", gotWhereClause, gotOrderAndLimitClause, gotArgs)
default:
t.Fatalf("unknown cmd: %s", d.Cmd)
}
return ""
})
}
168 changes: 168 additions & 0 deletions pkg/server/testdata/combined_statement_stats
Original file line number Diff line number Diff line change
@@ -0,0 +1,168 @@
# Try different limit values

query sort=SERVICE_LAT limit=1
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %'
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT DESC LIMIT $1
--ARGS--
[1]

query sort=SERVICE_LAT
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %'
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT DESC LIMIT $1
--ARGS--
[20000]

query sort=SERVICE_LAT limit=10000000
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %'
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT DESC LIMIT $1
--ARGS--
[10000000]


# Check all sort options

query sort=PCT_RUNTIME limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY ((statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT *
(statistics -> 'statistics' ->> 'cnt')::FLOAT) DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=SERVICE_LAT limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=CPU_TIME limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'execution_statistics' -> 'cpuSQLNanos' ->> 'mean')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=EXECUTION_COUNT limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' ->> 'cnt')::INT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=P99_STMTS_ONLY limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'latencyInfo' ->> 'p99')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=CONTENTION_TIME limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'execution_statistics' -> 'contentionTime' ->> 'mean')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=LATENCY_INFO_P50 limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'latencyInfo' ->> 'p50')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=LATENCY_INFO_P90 limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'latencyInfo' ->> 'p90')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=LATENCY_INFO_MIN limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'latencyInfo' ->> 'min')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=LATENCY_INFO_MAX limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' -> 'latencyInfo' ->> 'max')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=ROWS_PROCESSED limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY ((statistics -> 'statistics' -> 'rowsRead' ->> 'mean')::FLOAT +
(statistics -> 'statistics' -> 'rowsWritten' ->> 'mean')::FLOAT) DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=MAX_MEMORY limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'execution_statistics' -> 'maxMemUsage' ->> 'mean')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=NETWORK limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'execution_statistics' -> 'networkBytes' ->> 'mean')::FLOAT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=RETRIES limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' ->> 'maxRetries')::INT DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

query sort=LAST_EXEC limit=100 start=1 end=2
----
--WHERE--
WHERE app_name NOT LIKE '$ internal%' AND app_name NOT LIKE '$$ %' AND aggregated_ts >= $1 AND aggregated_ts <= $2
--ORDER AND LIMIT--
ORDER BY (statistics -> 'statistics' ->> 'lastExecAt') DESC LIMIT $3
--ARGS--
[1970-01-01 00:00:01 +0000 UTC 1970-01-01 00:00:02 +0000 UTC 100]

0 comments on commit a8636af

Please sign in to comment.