Skip to content

Commit 4c9b844

Browse files
craig[bot]kyle-a-wong
andcommitted
Merge #158365
158365: sql: Record execstats.QueryLevelStats for UDF / SP sub statements r=kyle-a-wong a=kyle-a-wong Fixes: [CRDB-57254](https://cockroachlabs.atlassian.net/browse/CRDB-57254) Epic: [CRDB-55081](https://cockroachlabs.atlassian.net/browse/CRDB-55081) Release note: None Co-authored-by: Kyle Wong <37189875+kyle-a-wong@users.noreply.github.com>
2 parents ca91a97 + ce73a11 commit 4c9b844

File tree

8 files changed

+43
-17
lines changed

8 files changed

+43
-17
lines changed

pkg/bench/rttanalysis/create_alter_role_bench_test.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -106,19 +106,19 @@ $$ LANGUAGE plpgsql;`
106106
},
107107
},
108108
{
109-
Name: "use 50 roles",
109+
Name: "use 25 roles",
110110
SetupEx: []string{
111111
"CREATE ROLE parent_role",
112-
createNRoles(50),
113-
grantRoleToNRoles("parent_role", 50),
112+
createNRoles(25),
113+
grantRoleToNRoles("parent_role", 25),
114114
"CREATE TABLE tab (a INT)",
115115
"GRANT SELECT ON tab TO parent_role",
116116
"INSERT INTO tab VALUES (1)",
117117
createFunc,
118118
},
119-
Stmt: "SELECT query_table_with_roles(50)",
119+
Stmt: "SELECT query_table_with_roles(25)",
120120
ResetEx: []string{
121-
dropNRoles(50),
121+
dropNRoles(25),
122122
"DROP ROLE parent_role",
123123
},
124124
},

pkg/bench/rttanalysis/testdata/benchmark_expectations

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -127,8 +127,8 @@ exp,benchmark
127127
17,Truncate/truncate_2_column_1_rows
128128
17,Truncate/truncate_2_column_2_rows
129129
0,UDFResolution/select_from_udf
130+
25,UseManyRoles/use_25_roles
130131
2,UseManyRoles/use_2_roles
131-
2,UseManyRoles/use_50_roles
132132
2,VirtualTableQueries/select_crdb_internal.invalid_objects_with_1_fk
133133
1,VirtualTableQueries/select_crdb_internal.tables_with_1_fk
134134
0,VirtualTableQueries/show_create_all_routines

pkg/sql/apply_join.go

Lines changed: 26 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,11 +12,14 @@ import (
1212
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
1313
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1414
"github.com/cockroachdb/cockroach/pkg/sql/distsql"
15+
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
1516
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec"
1617
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
1718
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
19+
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
1820
"github.com/cockroachdb/cockroach/pkg/sql/types"
1921
"github.com/cockroachdb/cockroach/pkg/util/tracing"
22+
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
2023
"github.com/cockroachdb/errors"
2124
"github.com/cockroachdb/redact"
2225
)
@@ -255,6 +258,7 @@ func (a *applyJoinNode) runNextRightSideIteration(params runParams, leftRow tree
255258
rowResultWriter := NewRowResultWriter(&a.run.rightRows)
256259
queryStats, err := runPlanInsidePlan(
257260
ctx, params, plan, rowResultWriter, nil /* deferredRoutineSender */, "", /* stmtForDistSQLDiagram */
261+
nil, /* sqlStatsBuilder */
258262
)
259263
if err != nil {
260264
return err
@@ -273,7 +277,8 @@ func runPlanInsidePlan(
273277
resultWriter rowResultWriter,
274278
deferredRoutineSender eval.DeferredRoutineSender,
275279
stmtForDistSQLDiagram string,
276-
) (topLevelQueryStats, error) {
280+
sqlStatsBuilder *sqlstats.RecordedStatementStatsBuilder,
281+
) (stats topLevelQueryStats, retErr error) {
277282
defer plan.close(ctx)
278283
execCfg := params.ExecCfg()
279284
recv := MakeDistSQLReceiver(
@@ -337,6 +342,26 @@ func runPlanInsidePlan(
337342
planCtx := execCfg.DistSQLPlanner.NewPlanningCtx(ctx, evalCtx, &plannerCopy, plannerCopy.txn, distributeType)
338343
planCtx.distSQLProhibitedErr = distSQLProhibitedErr
339344
planCtx.stmtType = recv.stmtType
345+
if sqlStatsBuilder != nil && plannerCopy.instrumentation.ShouldSaveFlows() {
346+
planCtx.collectExecStats = true
347+
planCtx.saveFlows = getDefaultSaveFlowsFunc(ctx, &plannerCopy, planComponentTypeInner)
348+
defer func() {
349+
sp := tracing.SpanFromContext(ctx)
350+
recording := sp.GetRecording(tracingpb.RecordingStructured)
351+
if recording.Len() == 0 {
352+
return
353+
}
354+
355+
var flowsMetadata []*execstats.FlowsMetadata
356+
for _, flowInfo := range plannerCopy.curPlan.distSQLFlowInfos {
357+
flowsMetadata = append(flowsMetadata, flowInfo.flowsMetadata)
358+
}
359+
queryLevelStats, err := execstats.GetQueryLevelStats(recording, false /* deterministicExplainAnalyze */, flowsMetadata)
360+
if err == nil {
361+
sqlStatsBuilder.ExecStats(&queryLevelStats)
362+
}
363+
}()
364+
}
340365
if params.p.innerPlansMustUseLeafTxn() {
341366
planCtx.flowConcurrency = distsql.ConcurrencyWithOuterPlan
342367
}

pkg/sql/plan.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -416,6 +416,7 @@ const (
416416
planComponentTypeMainQuery
417417
planComponentTypeSubquery
418418
planComponentTypePostquery
419+
planComponentTypeInner
419420
)
420421

421422
func (t planComponentType) String() string {

pkg/sql/recursive_cte.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -154,7 +154,7 @@ func (n *recursiveCTENode) Next(params runParams) (bool, error) {
154154
}
155155
queryStats, err := runPlanInsidePlan(
156156
planAndRunCtx, params, newPlan.(*planComponents), rowResultWriter(n),
157-
nil /* deferredRoutineSender */, "", /* stmtForDistSQLDiagram */
157+
nil /* deferredRoutineSender */, "" /* stmtForDistSQLDiagram */, nil, /* sqlStatsBuilder */
158158
)
159159
if err != nil {
160160
return false, err

pkg/sql/routine.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -409,7 +409,7 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
409409
}()
410410
}
411411
sqlstats.RecordStatementPhase(latencyRecorder, sqlstats.StatementStartExec)
412-
queryStats, err := runPlanInsidePlan(ctx, params, plan.(*planComponents), w, g, stmtForDistSQLDiagram)
412+
queryStats, err := runPlanInsidePlan(ctx, params, plan.(*planComponents), w, g, stmtForDistSQLDiagram, statsBuilder)
413413
sqlstats.RecordStatementPhase(latencyRecorder, sqlstats.StatementEndExec)
414414
if err != nil {
415415
statsBuilder.StatementError(err)

pkg/sql/sqlstats/testdata/plpgsql

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ CALL insert_incremental(5);
2020
show-stats db=udf_test app-name=plpgsql
2121
----
2222
{"count": "1", "fingerprint_id": "11c4e5400aa05722", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "CREATE OR REPLACE PROCEDURE insert_incremental(n INT8)\n\tLANGUAGE plpgsql\n\tAS $$_$$", "run_lat_not_zero": true, "sql_type": "TypeDDL", "summary": "CREATE OR REPLACE PROCEDURE insert_incremental(n INT8)\n\tLANGUAGE plpgsql\n\tAS $$_$$", "svc_lat_not_zero": true, "transaction_fingerprint_id": "3013cc07fc915da5"}
23-
{"count": "5", "fingerprint_id": "12d44a85acad2974", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": true, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT COALESCE(max(a), _) + _ FROM udf_test.public.test", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT COALESCE(max(a)...)... FROM udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "3013cc07fc915da5"}
23+
{"count": "5", "fingerprint_id": "12d44a85acad2974", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": true, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT COALESCE(max(a), _) + _ FROM udf_test.public.test", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT COALESCE(max(a)...)... FROM udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "3013cc07fc915da5"}
2424
{"count": "1", "fingerprint_id": "a82856550b801042", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "CALL insert_incremental(_)", "run_lat_not_zero": true, "sql_type": "TypeTCL", "summary": "CALL insert_incremental(_)", "svc_lat_not_zero": true, "transaction_fingerprint_id": "3013cc07fc915da5"}
25-
{"count": "5", "fingerprint_id": "a8eed74b07b66674", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "INSERT INTO udf_test.public.test(a) VALUES (next_val)", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "INSERT INTO udf_test.public.test(a)", "svc_lat_not_zero": true, "transaction_fingerprint_id": "3013cc07fc915da5"}
25+
{"count": "5", "fingerprint_id": "a8eed74b07b66674", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "INSERT INTO udf_test.public.test(a) VALUES (next_val)", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "INSERT INTO udf_test.public.test(a)", "svc_lat_not_zero": true, "transaction_fingerprint_id": "3013cc07fc915da5"}
2626
{"count": "1", "fingerprint_id": "f592c1a0850da0bf", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "CREATE TABLE IF NOT EXISTS test (a INT8)", "run_lat_not_zero": true, "sql_type": "TypeDDL", "summary": "CREATE TABLE IF NOT EXISTS test (a INT8)", "svc_lat_not_zero": true, "transaction_fingerprint_id": "5af17cec030c1760"}

pkg/sql/sqlstats/testdata/udf

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,10 @@ select do_something();
2121

2222
show-stats db=udf_test app-name=udf
2323
----
24-
{"count": "1", "fingerprint_id": "3e88643bc2cddb5f", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": true, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT max(a) FROM udf_test.public.test", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT max(a) FROM udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
25-
{"count": "1", "fingerprint_id": "4a5f350f5c6bf5ae", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "INSERT INTO udf_test.public.test (SELECT public.random_int())", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "INSERT INTO udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
24+
{"count": "1", "fingerprint_id": "3e88643bc2cddb5f", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": true, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT max(a) FROM udf_test.public.test", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT max(a) FROM udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
25+
{"count": "1", "fingerprint_id": "4a5f350f5c6bf5ae", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "INSERT INTO udf_test.public.test (SELECT public.random_int())", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "INSERT INTO udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
2626
{"count": "1", "fingerprint_id": "8be77c892ac9ad38", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "CREATE FUNCTION do_something()\n\tRETURNS INT8\n\tLANGUAGE SQL\n\tVOLATILE\n\tAS $$_$$", "run_lat_not_zero": true, "sql_type": "TypeDDL", "summary": "CREATE FUNCTION do_something()\n\tRETURNS INT8\n\tLANGUAGE SQL\n\tVOLATILE\n\tAS $$_$$", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
27-
{"count": "2", "fingerprint_id": "934cc9f967cd0f0f", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT floor(random() * _)::INT8", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT floor(random() ...)...", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
27+
{"count": "2", "fingerprint_id": "934cc9f967cd0f0f", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT floor(random() * _)::INT8", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT floor(random() ...)...", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
2828
{"count": "1", "fingerprint_id": "af9bcc145f0d85f3", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT do_something()", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT do_something()", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
2929
{"count": "1", "fingerprint_id": "bf1896e478760878", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT random_int()", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT random_int()", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1fe047693ba00214"}
3030
{"count": "1", "fingerprint_id": "f56262fdc29506d7", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": true, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "CREATE FUNCTION random_int()\n\tRETURNS INT8\n\tLANGUAGE SQL\n\tVOLATILE\n\tAS $$_$$", "run_lat_not_zero": true, "sql_type": "TypeDDL", "summary": "CREATE FUNCTION random_int()\n\tRETURNS INT8\n\tLANGUAGE SQL\n\tVOLATILE\n\tAS $$_$$", "svc_lat_not_zero": true, "transaction_fingerprint_id": "5a01dfb14494b108"}
@@ -39,7 +39,7 @@ COMMIT;
3939

4040
show-stats db=udf_test app-name=udf_transactions
4141
----
42-
{"count": "2", "fingerprint_id": "3e88643bc2cddb53", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": true, "plan_hash_set": true, "plan_implicit_txn": false, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT max(a) FROM udf_test.public.test", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT max(a) FROM udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1dc175ec90b4a99f"}
43-
{"count": "2", "fingerprint_id": "4a5f350f5c6bf5a2", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": false, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "INSERT INTO udf_test.public.test (SELECT public.random_int())", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "INSERT INTO udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1dc175ec90b4a99f"}
44-
{"count": "2", "fingerprint_id": "934cc9f967cd0f03", "nodes": [], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": false, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT floor(random() * _)::INT8", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT floor(random() ...)...", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1dc175ec90b4a99f"}
42+
{"count": "2", "fingerprint_id": "3e88643bc2cddb53", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": true, "plan_hash_set": true, "plan_implicit_txn": false, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT max(a) FROM udf_test.public.test", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT max(a) FROM udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1dc175ec90b4a99f"}
43+
{"count": "2", "fingerprint_id": "4a5f350f5c6bf5a2", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": false, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "INSERT INTO udf_test.public.test (SELECT public.random_int())", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "INSERT INTO udf_test.public.test", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1dc175ec90b4a99f"}
44+
{"count": "2", "fingerprint_id": "934cc9f967cd0f03", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": false, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT floor(random() * _)::INT8", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT floor(random() ...)...", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1dc175ec90b4a99f"}
4545
{"count": "2", "fingerprint_id": "af9bcc145f0d85ff", "nodes": [1], "parse_lat_not_zero": true, "plan_distributed": false, "plan_full_scan": false, "plan_hash_set": true, "plan_implicit_txn": false, "plan_lat_not_zero": true, "plan_vectorized": true, "query": "SELECT do_something()", "run_lat_not_zero": true, "sql_type": "TypeDML", "summary": "SELECT do_something()", "svc_lat_not_zero": true, "transaction_fingerprint_id": "1dc175ec90b4a99f"}

0 commit comments

Comments
 (0)