From 00cc737ad8382e686ed0a761fb569014ca6751e6 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Thu, 2 Feb 2023 07:54:51 -0500 Subject: [PATCH 1/4] multitenant: fix createInMemoryTenant() roachtest util bug Release note: None Epic: None --- pkg/cmd/roachtest/tests/multitenant_utils.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/multitenant_utils.go b/pkg/cmd/roachtest/tests/multitenant_utils.go index f604b530e546..686feb9d78a7 100644 --- a/pkg/cmd/roachtest/tests/multitenant_utils.go +++ b/pkg/cmd/roachtest/tests/multitenant_utils.go @@ -354,7 +354,7 @@ func createInMemoryTenant( // server in a retry loop. var tenantSQL *sqlutils.SQLRunner testutils.SucceedsSoon(t, func() error { - tenantConn, err := c.ConnE(ctx, t.L(), nodes.RandNode()[0]) + tenantConn, err := c.ConnE(ctx, t.L(), nodes.RandNode()[0], option.TenantName(tenantName)) if err != nil { return err } From 18c1540cdece8611861f6ec8b7fc8f2a99f1b8e8 Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Wed, 1 Feb 2023 13:40:23 -0500 Subject: [PATCH 2/4] sql: delete descriptor when function is dropped Previously, in legacy schema changer, descriptor stayed as a orphan recrod in system.descriptor when a function is dropped. While this is not a big issue, we should keep data clean. This PR adds schema change job logic to handle the deletions. Fixes: #95364 Release note: None --- pkg/jobs/jobspb/jobs.proto | 4 ++ pkg/sql/drop_database.go | 1 + pkg/sql/drop_function.go | 27 ++++++++- pkg/sql/drop_function_test.go | 2 +- pkg/sql/drop_schema.go | 8 +++ pkg/sql/logictest/testdata/logic_test/udf | 74 +++++++++++++++++++++++ pkg/sql/schema_changer.go | 46 ++++++++++++-- pkg/sql/table.go | 7 +++ 8 files changed, 162 insertions(+), 7 deletions(-) diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 6429362a3417..5f73be525b5b 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -805,6 +805,9 @@ message SchemaChangeDetails { (gogoproto.customname) = "DroppedDatabaseID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" ]; + // dropped_functions holds the set of functions to drop as part of DROP + // FUNCTION, DROP DATABASE or DROP SCHEMA statement. + repeated uint32 dropped_functions = 12 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; // desc_id is the target descriptor for this schema change. Note that this ID // is not always a table ID! We allow referencing any descriptor here to allow // generic schema changes on descriptors whose schema change process involves @@ -832,6 +835,7 @@ message SchemaChangeDetails { (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" ]; + // Next id 13. } message SchemaChangeProgress { diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 190defc9bb24..cb19c9d0f0b6 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -179,6 +179,7 @@ func (n *dropDatabaseNode) startExec(params runParams) error { schemasIDsToDelete, n.d.getDroppedTableDetails(), n.d.typesToDelete, + n.d.functionsToDelete, tree.AsStringWithFQNames(n.n, params.Ann()), ) diff --git a/pkg/sql/drop_function.go b/pkg/sql/drop_function.go index 943f6ec9fd83..e05896cea515 100644 --- a/pkg/sql/drop_function.go +++ b/pkg/sql/drop_function.go @@ -14,6 +14,8 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" @@ -22,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/intsets" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" ) @@ -223,7 +226,7 @@ func (p *planner) dropFunctionImpl(ctx context.Context, fnMutable *funcdesc.Muta // Mark the UDF as dropped. fnMutable.SetDropped() - if err := p.writeFuncSchemaChange(ctx, fnMutable); err != nil { + if err := p.writeDropFuncSchemaChange(ctx, fnMutable); err != nil { return err } fnName := tree.MakeQualifiedFunctionName(p.CurrentDatabase(), scDesc.GetName(), fnMutable.GetName()) @@ -245,6 +248,28 @@ func (p *planner) writeFuncSchemaChange(ctx context.Context, funcDesc *funcdesc. return p.writeFuncDesc(ctx, funcDesc) } +func (p *planner) writeDropFuncSchemaChange(ctx context.Context, funcDesc *funcdesc.Mutable) error { + _, recordExists := p.extendedEvalCtx.jobs.uniqueToCreate[funcDesc.ID] + if recordExists { + // For now being, we create jobs for functions only when functions are + // dropped. + return nil + } + jobRecord := jobs.Record{ + JobID: p.extendedEvalCtx.ExecCfg.JobRegistry.MakeJobID(), + Description: "Drop Function", + Username: p.User(), + DescriptorIDs: descpb.IDs{funcDesc.ID}, + Details: jobspb.SchemaChangeDetails{ + DroppedFunctions: descpb.IDs{funcDesc.ID}, + }, + Progress: jobspb.TypeSchemaChangeProgress{}, + } + p.extendedEvalCtx.jobs.uniqueToCreate[funcDesc.ID] = &jobRecord + log.Infof(ctx, "queued drop function job %d for function %d", jobRecord.JobID, funcDesc.ID) + return p.writeFuncDesc(ctx, funcDesc) +} + func (p *planner) removeDependentFunction( ctx context.Context, tbl *tabledesc.Mutable, fn *funcdesc.Mutable, ) error { diff --git a/pkg/sql/drop_function_test.go b/pkg/sql/drop_function_test.go index ee3032daa94b..d7e697980c66 100644 --- a/pkg/sql/drop_function_test.go +++ b/pkg/sql/drop_function_test.go @@ -398,7 +398,7 @@ $$; err = sql.TestingDescsTxn(ctx, s, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { _, err := col.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Function(ctx, 113) require.Error(t, err) - require.Regexp(t, "descriptor is being dropped", err.Error()) + require.Regexp(t, "function undefined", err.Error()) return nil }) require.NoError(t, err) diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index 6254a5a51aa1..b309502a7225 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -165,6 +166,7 @@ func (n *dropSchemaNode) startExec(params runParams) error { schemaIDs, n.d.getDroppedTableDetails(), n.d.typesToDelete, + n.d.functionsToDelete, tree.AsStringWithFQNames(n.n, params.Ann()), ) @@ -230,12 +232,17 @@ func (p *planner) createDropSchemaJob( schemas []descpb.ID, tableDropDetails []jobspb.DroppedTableDetails, typesToDrop []*typedesc.Mutable, + functionsToDrop []*funcdesc.Mutable, jobDesc string, ) { typeIDs := make([]descpb.ID, 0, len(typesToDrop)) for _, t := range typesToDrop { typeIDs = append(typeIDs, t.ID) } + fnIDs := make([]descpb.ID, 0, len(functionsToDrop)) + for _, f := range functionsToDrop { + fnIDs = append(fnIDs, f.ID) + } p.extendedEvalCtx.QueueJob(&jobs.Record{ Description: jobDesc, @@ -245,6 +252,7 @@ func (p *planner) createDropSchemaJob( DroppedSchemas: schemas, DroppedTables: tableDropDetails, DroppedTypes: typeIDs, + DroppedFunctions: fnIDs, DroppedDatabaseID: descpb.InvalidID, // The version distinction for database jobs doesn't matter for jobs that // drop schemas. diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index e540adda97a1..86ea143a5c0e 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -2987,3 +2987,77 @@ query I SELECT f96326(); ---- 1 + +subtest regression_95364 + +statement ok +CREATE FUNCTION f_95364() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +let $dropped_fn_id +SELECT function_id FROM crdb_internal.create_function_statements WHERE function_name = 'f_95364'; + +query I +SELECT count(descriptor) FROM system.descriptor WHERE id = $dropped_fn_id; +---- +1 + +statement ok +DROP FUNCTION f_95364; + +query I +SELECT count(descriptor) FROM system.descriptor WHERE id = $dropped_fn_id; +---- +0 + +statement ok +CREATE DATABASE db_95364; + +statement ok +USE db_95364; + +statement ok +CREATE FUNCTION f_95364_2() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +let $dropped_fn_id +SELECT function_id FROM crdb_internal.create_function_statements WHERE function_name = 'f_95364_2'; + +query I +SELECT count(descriptor) FROM system.descriptor WHERE id = $dropped_fn_id; +---- +1 + +statement ok +USE test; + +statement ok +DROP DATABASE db_95364 CASCADE; + +query I +SELECT count(descriptor) FROM system.descriptor WHERE id = $dropped_fn_id; +---- +0 + +statement ok +USE test; + +statement ok +CREATE SCHEMA sc_95364; + +statement ok +CREATE FUNCTION sc_95364.f_95364_3() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +let $dropped_fn_id +SELECT function_id FROM crdb_internal.create_function_statements WHERE function_name = 'f_95364_3'; + +query I +SELECT count(descriptor) FROM system.descriptor WHERE id = $dropped_fn_id; +---- +1 + +statement ok +DROP SCHEMA sc_95364 CASCADE; + +query I +SELECT count(descriptor) FROM system.descriptor WHERE id = $dropped_fn_id; +---- +0 diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index e7066583bf36..86700ff201d9 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -109,6 +109,7 @@ type SchemaChanger struct { mutationID descpb.MutationID droppedDatabaseID descpb.ID droppedSchemaIDs catalog.DescriptorIDSet + droppedFnIDs catalog.DescriptorIDSet sqlInstanceID base.SQLInstanceID leaseMgr *lease.Manager db isql.DB @@ -690,6 +691,10 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { if sc.droppedDatabaseID != desc.GetID() { return nil } + case catalog.FunctionDescriptor: + if !sc.droppedFnIDs.Contains(desc.GetID()) { + return nil + } default: return nil } @@ -2568,11 +2573,12 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er return err } } - execSchemaChange := func(descID descpb.ID, mutationID descpb.MutationID, droppedDatabaseID descpb.ID, droppedSchemaIDs descpb.IDs) error { + execSchemaChange := func(descID descpb.ID, mutationID descpb.MutationID, droppedDatabaseID descpb.ID, droppedSchemaIDs descpb.IDs, droppedFnIDs descpb.IDs) error { sc := SchemaChanger{ descID: descID, mutationID: mutationID, droppedSchemaIDs: catalog.MakeDescriptorIDSet(droppedSchemaIDs...), + droppedFnIDs: catalog.MakeDescriptorIDSet(droppedFnIDs...), droppedDatabaseID: droppedDatabaseID, sqlInstanceID: p.ExecCfg().NodeInfo.NodeID.SQLInstanceID(), db: p.ExecCfg().InternalDB, @@ -2669,14 +2675,38 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er // Drop the child tables. for i := range details.DroppedTables { droppedTable := &details.DroppedTables[i] - if err := execSchemaChange(droppedTable.ID, descpb.InvalidMutationID, details.DroppedDatabaseID, details.DroppedSchemas); err != nil { + if err := execSchemaChange( + droppedTable.ID, + descpb.InvalidMutationID, + details.DroppedDatabaseID, + details.DroppedSchemas, + nil, /* droppedFnIDs */ + ); err != nil { return err } } // Drop all schemas. for _, id := range details.DroppedSchemas { - if err := execSchemaChange(id, descpb.InvalidMutationID, descpb.InvalidID, details.DroppedSchemas); err != nil { + if err := execSchemaChange( + id, + descpb.InvalidMutationID, + descpb.InvalidID, + details.DroppedSchemas, + nil, /* droppedFnIDs */ + ); err != nil { + return err + } + } + + for _, id := range details.DroppedFunctions { + if err := execSchemaChange( + id, + descpb.InvalidMutationID, + descpb.InvalidID, + nil, /* droppedSchemaIDs */ + details.DroppedFunctions, + ); err != nil { return err } } @@ -2684,7 +2714,13 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er // Drop the database, if applicable. if details.FormatVersion >= jobspb.DatabaseJobFormatVersion { if dbID := details.DroppedDatabaseID; dbID != descpb.InvalidID { - if err := execSchemaChange(dbID, descpb.InvalidMutationID, details.DroppedDatabaseID, details.DroppedSchemas); err != nil { + if err := execSchemaChange( + dbID, + descpb.InvalidMutationID, + details.DroppedDatabaseID, + details.DroppedSchemas, + nil, /* droppedFnIDs */ + ); err != nil { return err } // If there are no tables to GC, the zone config needs to be deleted now. @@ -2737,7 +2773,7 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er // schema changer. This can be any single-table schema change or any change to // a database or schema other than a drop. if details.DescID != descpb.InvalidID { - return execSchemaChange(details.DescID, details.TableMutationID, details.DroppedDatabaseID, details.DroppedSchemas) + return execSchemaChange(details.DescID, details.TableMutationID, details.DroppedDatabaseID, details.DroppedSchemas, nil /* droppedFnIDs */) } return nil } diff --git a/pkg/sql/table.go b/pkg/sql/table.go index c5cc7a17ae4b..69a65cca1e1e 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catsessiondata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" @@ -42,6 +43,7 @@ func (p *planner) createDropDatabaseJob( schemasToDrop []descpb.ID, tableDropDetails []jobspb.DroppedTableDetails, typesToDrop []*typedesc.Mutable, + functionsToDrop []*funcdesc.Mutable, jobDesc string, ) { // TODO (lucy): This should probably be deleting the queued jobs for all the @@ -54,6 +56,10 @@ func (p *planner) createDropDatabaseJob( for _, t := range typesToDrop { typeIDs = append(typeIDs, t.ID) } + funcIDs := make([]descpb.ID, 0, len(functionsToDrop)) + for _, t := range functionsToDrop { + funcIDs = append(funcIDs, t.ID) + } jobRecord := &jobs.Record{ Description: jobDesc, Username: p.User(), @@ -62,6 +68,7 @@ func (p *planner) createDropDatabaseJob( DroppedSchemas: schemasToDrop, DroppedTables: tableDropDetails, DroppedTypes: typeIDs, + DroppedFunctions: funcIDs, DroppedDatabaseID: databaseID, FormatVersion: jobspb.DatabaseJobFormatVersion, }, From 0a8e8005a7b21980c281d94cfa79b647add04d0b Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Wed, 1 Feb 2023 17:09:08 -0500 Subject: [PATCH 3/4] upgrades: add upgrade to delete dropped function descriptors This commit adds an upgrade to delete orphan descriptors of functions dropped through legacy schema changer before 23.1. Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 6 + pkg/sql/schema_changer.go | 11 + pkg/testutils/sqlutils/inject.go | 6 + pkg/upgrade/upgrades/BUILD.bazel | 3 + ...delete_descriptors_of_dropped_functions.go | 84 +++++++ ...e_descriptors_of_dropped_functions_test.go | 228 ++++++++++++++++++ pkg/upgrade/upgrades/upgrades.go | 5 + 9 files changed, 345 insertions(+), 2 deletions(-) create mode 100644 pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions.go create mode 100644 pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 06ef5777dc8b..aa505b41e955 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -298,4 +298,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 1000022.2-34 set the active cluster version in the format '.' +version version 1000022.2-36 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 4c3434db6dff..6a0d139de122 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -236,6 +236,6 @@
trace.opentelemetry.collector
stringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. -
version
version1000022.2-34set the active cluster version in the format '<major>.<minor>' +
version
version1000022.2-36set the active cluster version in the format '<major>.<minor>' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 160d8d30af71..69249a29b130 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -408,6 +408,8 @@ const ( // columnar scans in the KV layer. V23_1_KVDirectColumnarScans + V23_1_DeleteDroppedFunctionDescriptors + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -705,6 +707,10 @@ var rawVersionsSingleton = keyedVersions{ Key: V23_1_KVDirectColumnarScans, Version: roachpb.Version{Major: 22, Minor: 2, Internal: 34}, }, + { + Key: V23_1_DeleteDroppedFunctionDescriptors, + Version: roachpb.Version{Major: 22, Minor: 2, Internal: 36}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 86700ff201d9..68f5ab736115 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -634,6 +634,17 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { // Pull out the requested descriptor. desc, err := sc.getTargetDescriptor(ctx) if err != nil { + // We had a bug where function descriptors are not deleted after DROP + // FUNCTION in legacy schema changer (see #95364). We then add logic to + // handle the deletion in jobs and added upgrades to delete all dropped + // functions. It's possible that such job is resumed after the cluster is + // upgraded (all dropped descriptors are deleted), and we would fail to find + // the descriptor here. In this case, we can simply assume the job is done + // since we only handle descriptor deletes for functions and `droppedFnIDs` + // is not empty only when dropping functions. + if sc.droppedFnIDs.Len() > 0 && errors.Is(err, catalog.ErrDescriptorNotFound) { + return nil + } return err } diff --git a/pkg/testutils/sqlutils/inject.go b/pkg/testutils/sqlutils/inject.go index cf882b784986..fb051a3e517e 100644 --- a/pkg/testutils/sqlutils/inject.go +++ b/pkg/testutils/sqlutils/inject.go @@ -101,6 +101,10 @@ func InjectDescriptors( } // Inject the namespace entries. for _, d := range others { + if d.GetFunction() != nil { + // Functions doesn't have namespace entry. + continue + } id, _, name, _, err := descpb.GetDescriptorMetadata(d) if err != nil { return err @@ -145,5 +149,7 @@ func resetVersionAndModificationTime(d *descpb.Descriptor) { d.Type.Version = 1 case *descpb.Descriptor_Table: d.Table.Version = 1 + case *descpb.Descriptor_Function: + d.Function.Version = 1 } } diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 34ebb1d27d46..5fd374a39dcf 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "alter_statement_statistics_index_recommendations.go", "alter_table_statistics_partial_predicate_and_id.go", "create_index_usage_statement_statistics.go", + "delete_descriptors_of_dropped_functions.go", "desc_id_sequence_for_system_tenant.go", "descriptor_utils.go", "ensure_sql_schema_telemetry_schedule.go", @@ -89,6 +90,7 @@ go_test( "alter_table_statistics_partial_predicate_and_id_test.go", "builtins_test.go", "create_index_usage_statement_statistics_test.go", + "delete_descriptors_of_dropped_functions_test.go", "desc_id_sequence_for_system_tenant_test.go", "descriptor_utils_test.go", "ensure_sql_schema_telemetry_schedule_test.go", @@ -134,6 +136,7 @@ go_test( "//pkg/sql/catalog/descbuilder", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/funcdesc", "//pkg/sql/catalog/schemadesc", "//pkg/sql/catalog/schematelemetry/schematelemetrycontroller", "//pkg/sql/catalog/systemschema", diff --git a/pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions.go b/pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions.go new file mode 100644 index 000000000000..b53de459ffab --- /dev/null +++ b/pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions.go @@ -0,0 +1,84 @@ +// Copyright 2023 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 upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +const deleteDroppedFunctionQuery = ` +WITH to_json AS ( + SELECT + id, + crdb_internal.pb_to_json( + 'cockroach.sql.sqlbase.Descriptor', + descriptor, + false + ) AS d + FROM + system.descriptor +), +to_delete AS ( + SELECT id + FROM to_json + WHERE + d->'function' IS NOT NULL + AND d->'function'->>'declarativeSchemaChangerState' IS NULL + AND d->'function'->>'state' = 'DROP' +) +SELECT crdb_internal.unsafe_delete_descriptor(id, false) +FROM to_delete +WHERE id >= $1 +AND id < $2; +` + +func deleteDescriptorsOfDroppedFunctions( + ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + if err := d.DB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error { + row, err := txn.QueryRow( + ctx, + "upgrade-delete-dropped-function-descriptors-get-max-descriptor-id", + txn.KV(), + `SELECT max(id) FROM system.descriptor`, + ) + + if err != nil { + return err + } + + maxID := int(tree.MustBeDInt(row[0])) + const batchSize = 50 + + for curID := 1; curID <= maxID; curID += batchSize { + _, err := txn.Exec( + ctx, + "upgrade-delete-dropped-function-descriptors", /* opName */ + txn.KV(), + deleteDroppedFunctionQuery, + curID, + curID+batchSize, + ) + if err != nil { + return err + } + } + return nil + }); err != nil { + return err + } + return nil +} diff --git a/pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions_test.go b/pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions_test.go new file mode 100644 index 000000000000..d748513da36f --- /dev/null +++ b/pkg/upgrade/upgrades/delete_descriptors_of_dropped_functions_test.go @@ -0,0 +1,228 @@ +// Copyright 2023 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 upgrades_test + +import ( + "context" + gosql "database/sql" + "encoding/hex" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descbuilder" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +const countToDeleteFunctionQuery = ` +WITH to_json AS ( + SELECT + id, + crdb_internal.pb_to_json( + 'cockroach.sql.sqlbase.Descriptor', + descriptor, + false + ) AS d + FROM + system.descriptor +), +to_delete AS ( + SELECT id + FROM to_json + WHERE + d->'function' IS NOT NULL + AND d->'function'->>'declarativeSchemaChangerState' IS NULL + AND d->'function'->>'state' = 'DROP' +) +SELECT count(id) +FROM to_delete +` + +// Similar to countToDeleteFunctionQuery but includes functions dropped by +// declarative schema changer. +const countTotalDroppedFunctionQuery = ` +WITH to_json AS ( + SELECT + id, + crdb_internal.pb_to_json( + 'cockroach.sql.sqlbase.Descriptor', + descriptor, + false + ) AS d + FROM + system.descriptor +), +to_delete AS ( + SELECT id + FROM to_json + WHERE + d->'function' IS NOT NULL + AND d->'function'->>'state' = 'DROP' +) +SELECT count(id) +FROM to_delete +` + +// Similar to countToDeleteFunctionQuery but includes functions dropped by +// declarative schema changer. +const countTotalFunctionQuery = ` +WITH to_json AS ( + SELECT + id, + crdb_internal.pb_to_json( + 'cockroach.sql.sqlbase.Descriptor', + descriptor, + false + ) AS d + FROM + system.descriptor +), +to_delete AS ( + SELECT id + FROM to_json + WHERE + d->'function' IS NOT NULL +) +SELECT count(id) +FROM to_delete +` + +func TestDeleteDescriptorsOfDroppedFunctions(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var ( + v0 = clusterversion.ByKey(clusterversion.V23_1_DeleteDroppedFunctionDescriptors - 1) + v1 = clusterversion.ByKey(clusterversion.V23_1_DeleteDroppedFunctionDescriptors) + ) + + droppedFunctionsHex := []string{ + "2a5d0a016610731864206532100a0c08011040180030005014600010003801420953454c45435420313b48015000580162210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418028001028a01009001029a0100", + "2a5d0a016710741864206532100a0c08011040180030005014600010003801420953454c45435420313b48015000580162210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418028001028a01009001029a0100", + "2a5d0a016610751864206532100a0c08011040180030005014600010003801420953454c45435420313b48015000580162210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418028001028a01009001029a0100", + "2a5d0a016710761864206532100a0c08011040180030005014600010003801420953454c45435420313b48015000580162210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418028001028a01009001029a0100", + } + + droppedFunctionsHexDeclarative := "2ae5030a04666e6577106c1864206532100a0c08011040180030005014600010003801420953454c45435420313b48015000580162210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418028001028a01009001029a0100a20181030a150a0bea0308086c1204726f6f7412040801100118010a1a0a10f2030d086c120561646d696e1802200212040801100118010a190a0ff2030c086c1204726f6f741802200212040801100118010a1e0a144a12086c220e0a0c08011040180030005014600012040801100118010a110a07a20604086c106512040801100118010a150a0b820a08086c1204666e657712040801100118010a130a098a0a06086c1202080112040801100118010a0f0a05920a02086c12040801100118010a130a099a0a06086c5a02080112040801100118010a1e0a14a20a11086c120953454c45435420313b1a020801120408011001180112510a4f0a1244524f502046554e4354494f4e20666e6577122a44524f502046554e4354494f4e20e280b92222e280ba2ee280b92222e280ba2ee280b9666e6577e280ba1a0d44524f502046554e4354494f4e1a170a04726f6f74120f2420636f636b726f6163682073716c220a01010105010101010101288180de85ed86c7cd0b320a00010203040506070809" + + publicFunctionHex := "2a640a08667075626c696335106d1864206532100a0c08011040180030005014600010003801420953454c45435420313b48015000580162210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418028001008a01009001019a0100" + + ctx := context.Background() + settings := cluster.MakeTestingClusterSettingsWithVersions(v1, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV)) + + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: v0, + }, + }, + }, + }) + + defer tc.Stopper().Stop(ctx) + + sqlDB := tc.ServerConn(0) + tdb := sqlutils.MakeSQLRunner(sqlDB) + + var parentID, parentSchemaID descpb.ID + tdb.Exec(t, "CREATE TABLE temp_tbl()") + tdb.QueryRow(t, `SELECT "parentID", "parentSchemaID" FROM system.namespace WHERE name = 'temp_tbl'`). + Scan(&parentID, &parentSchemaID) + + for _, fnHex := range droppedFunctionsHex { + decodeFunctionDescriptorAndInsert(t, ctx, sqlDB, fnHex, parentID, parentSchemaID, true /* dropped */) + } + decodeFunctionDescriptorAndInsert(t, ctx, sqlDB, droppedFunctionsHexDeclarative, parentID, parentSchemaID, true /* dropped */) + decodeFunctionDescriptorAndInsert(t, ctx, sqlDB, publicFunctionHex, parentID, parentSchemaID, false /* dropped */) + + // Make sure that the number of function descriptors to delete is right. + row := tdb.QueryRow(t, countToDeleteFunctionQuery) + var cntFnToDelete int + row.Scan(&cntFnToDelete) + require.Equal(t, 4, cntFnToDelete) + + row = tdb.QueryRow(t, countTotalDroppedFunctionQuery) + row.Scan(&cntFnToDelete) + require.Equal(t, 5, cntFnToDelete) + + row = tdb.QueryRow(t, countTotalFunctionQuery) + row.Scan(&cntFnToDelete) + require.Equal(t, 6, cntFnToDelete) + + var originalTotalDescCnt int + row = tdb.QueryRow(t, `SELECT count(*) FROM system.descriptor`) + row.Scan(&originalTotalDescCnt) + + // Upgrade to the new cluster version. + tdb.Exec(t, `SET CLUSTER SETTING version = $1`, v1.String()) + tdb.CheckQueryResultsRetry(t, "SHOW CLUSTER SETTING version", + [][]string{{v1.String()}}) + + // Make sure there is nothing to delete. + row = tdb.QueryRow(t, countToDeleteFunctionQuery) + row.Scan(&cntFnToDelete) + require.Equal(t, 0, cntFnToDelete) + + // Make sure other descriptors are intact. + var newTotalDescCnt int + row = tdb.QueryRow(t, `SELECT count(*) FROM system.descriptor`) + row.Scan(&newTotalDescCnt) + require.Equal(t, originalTotalDescCnt-4, newTotalDescCnt) + + row = tdb.QueryRow(t, countTotalDroppedFunctionQuery) + row.Scan(&cntFnToDelete) + require.Equal(t, 1, cntFnToDelete) + + row = tdb.QueryRow(t, countTotalFunctionQuery) + row.Scan(&cntFnToDelete) + require.Equal(t, 2, cntFnToDelete) +} + +func decodeFunctionDescriptorAndInsert( + t *testing.T, + ctx context.Context, + sqlDB *gosql.DB, + hexEncodedDescriptor string, + parentID, parentSchemaID descpb.ID, + dropped bool, +) { + decodedDescriptor, err := hex.DecodeString(hexEncodedDescriptor) + require.NoError(t, err) + b, err := descbuilder.FromBytesAndMVCCTimestamp(decodedDescriptor, hlc.Timestamp{WallTime: 1}) + require.NoError(t, err) + require.NotNil(t, b) + require.Equal(t, catalog.Function, b.DescriptorType()) + // Run post deserialization changes. + require.NoError(t, b.RunPostDeserializationChanges()) + // Modify this descriptor's parentID and parentSchemaID + fnDesc := b.(funcdesc.FunctionDescriptorBuilder).BuildCreatedMutableFunction() + fnDesc.ParentID = parentID + fnDesc.ParentSchemaID = parentSchemaID + require.Equal(t, dropped, fnDesc.Dropped()) + // Insert the descriptor into test cluster. + require.NoError(t, sqlutils.InjectDescriptors( + ctx, sqlDB, []*descpb.Descriptor{fnDesc.DescriptorProto()}, true, /* force */ + )) +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 5bc1774b02ff..4d5e8a273e5e 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -258,6 +258,11 @@ var upgrades = []upgradebase.Upgrade{ keyVisualizerTablesMigration, "initialize key visualizer tables and jobs", ), + upgrade.NewTenantUpgrade("delete descriptors of dropped functions", + toCV(clusterversion.V23_1_DeleteDroppedFunctionDescriptors), + upgrade.NoPrecondition, + deleteDescriptorsOfDroppedFunctions, + ), } func init() { From 6eb85d757b05ba3960b74a8d8fbc3e375b7954c6 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 1 Feb 2023 18:33:05 -0800 Subject: [PATCH 4/4] appstatspb: extract app_stats from roachpb into new package Epic: None Release note: None --- BUILD.bazel | 1 - docs/generated/http/BUILD.bazel | 1 + docs/generated/sql/functions.md | 4 +- pkg/BUILD.bazel | 4 + pkg/ccl/serverccl/statusccl/BUILD.bazel | 1 + .../serverccl/statusccl/tenant_status_test.go | 3 +- pkg/gen/protobuf.bzl | 1 + pkg/roachpb/BUILD.bazel | 3 - pkg/server/BUILD.bazel | 2 + pkg/server/combined_statement_stats.go | 26 ++--- .../diagnostics/diagnosticspb/BUILD.bazel | 2 + .../diagnosticspb/diagnostics.proto | 2 +- pkg/server/drain_test.go | 4 +- pkg/server/serverpb/BUILD.bazel | 2 + pkg/server/serverpb/status.proto | 6 +- pkg/server/statements.go | 5 +- pkg/server/stats_test.go | 4 +- pkg/server/status_test.go | 9 +- pkg/sql/BUILD.bazel | 2 + pkg/sql/appstatspb/BUILD.bazel | 44 +++++++++ pkg/{roachpb => sql/appstatspb}/app_stats.go | 2 +- .../appstatspb}/app_stats.proto | 2 +- .../appstatspb}/app_stats_test.go | 2 +- pkg/sql/conn_executor.go | 27 +++--- pkg/sql/conn_executor_exec.go | 9 +- pkg/sql/contention/BUILD.bazel | 2 + pkg/sql/contention/event_store_test.go | 9 +- pkg/sql/contention/resolver.go | 22 ++--- pkg/sql/contention/resolver_test.go | 16 ++-- pkg/sql/contention/test_utils.go | 10 +- pkg/sql/contention/txnidcache/BUILD.bazel | 4 +- pkg/sql/contention/txnidcache/fifo_cache.go | 8 +- .../contention/txnidcache/fifo_cache_test.go | 10 +- pkg/sql/contention/txnidcache/txn_id_cache.go | 10 +- .../txnidcache/txn_id_cache_test.go | 22 ++--- pkg/sql/contention/txnidcache/writer_test.go | 4 +- pkg/sql/contentionpb/BUILD.bazel | 1 + pkg/sql/contentionpb/contention.proto | 10 +- pkg/sql/crdb_internal.go | 17 ++-- pkg/sql/exec_log.go | 6 +- pkg/sql/exec_util.go | 7 +- pkg/sql/executor_statement_metrics.go | 6 +- pkg/sql/instrumentation.go | 4 +- pkg/sql/instrumentation_test.go | 20 ++-- pkg/sql/opt/exec/explain/BUILD.bazel | 1 + pkg/sql/opt/exec/explain/output.go | 14 +-- pkg/sql/sem/builtins/BUILD.bazel | 1 + pkg/sql/sem/builtins/builtins.go | 19 ++-- pkg/sql/sqlstats/BUILD.bazel | 1 + pkg/sql/sqlstats/insights/BUILD.bazel | 4 +- pkg/sql/sqlstats/insights/detector.go | 8 +- pkg/sql/sqlstats/insights/detector_test.go | 4 +- pkg/sql/sqlstats/insights/insights.proto | 4 +- pkg/sql/sqlstats/insights/insights_test.go | 4 +- pkg/sql/sqlstats/insights/registry_test.go | 20 ++-- .../sqlstats/persistedsqlstats/BUILD.bazel | 3 +- .../sqlstats/persistedsqlstats/appStats.go | 10 +- .../persistedsqlstats/combined_iterator.go | 18 ++-- pkg/sql/sqlstats/persistedsqlstats/flush.go | 28 +++--- .../sqlstats/persistedsqlstats/flush_test.go | 5 +- .../persistedsqlstats/mem_iterator.go | 10 +- .../sqlstats/persistedsqlstats/reader_test.go | 14 +-- .../sqlstatsutil/BUILD.bazel | 4 +- .../sqlstatsutil/json_decoding.go | 32 ++++--- .../sqlstatsutil/json_encoding.go | 30 +++--- .../sqlstatsutil/json_encoding_test.go | 78 +++++++-------- .../sqlstatsutil/json_impl.go | 26 ++--- .../sqlstatsutil/testutils.go | 6 +- .../sqlstats/persistedsqlstats/stmt_reader.go | 10 +- .../sqlstats/persistedsqlstats/txn_reader.go | 8 +- pkg/sql/sqlstats/sslocal/BUILD.bazel | 4 +- pkg/sql/sqlstats/sslocal/iterator_test.go | 10 +- pkg/sql/sqlstats/sslocal/sql_stats_test.go | 94 +++++++++---------- pkg/sql/sqlstats/sslocal/sslocal_iterator.go | 10 +- .../sslocal/sslocal_stats_collector.go | 10 +- pkg/sql/sqlstats/sslocal/temp_sql_stats.go | 2 +- pkg/sql/sqlstats/ssmemstorage/BUILD.bazel | 1 + .../sqlstats/ssmemstorage/ss_mem_iterator.go | 20 ++-- .../sqlstats/ssmemstorage/ss_mem_storage.go | 66 ++++++------- .../sqlstats/ssmemstorage/ss_mem_writer.go | 11 ++- pkg/sql/sqlstats/ssmemstorage/utils.go | 4 +- pkg/sql/sqlstats/ssprovider.go | 21 +++-- pkg/sql/sqltestutils/BUILD.bazel | 1 + pkg/sql/sqltestutils/telemetry.go | 6 +- pkg/sql/telemetry_logging_test.go | 4 +- pkg/sql/txn_fingerprint_id_cache.go | 12 +-- pkg/sql/txn_fingerprint_id_cache_test.go | 8 +- .../src/util/appStats/appStats.spec.ts | 2 +- 88 files changed, 544 insertions(+), 460 deletions(-) create mode 100644 pkg/sql/appstatspb/BUILD.bazel rename pkg/{roachpb => sql/appstatspb}/app_stats.go (99%) rename pkg/{roachpb => sql/appstatspb}/app_stats.proto (99%) rename pkg/{roachpb => sql/appstatspb}/app_stats_test.go (99%) diff --git a/BUILD.bazel b/BUILD.bazel index 50aa5dead6a2..172d3f476c00 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -61,7 +61,6 @@ exports_files([ # See pkg/roachpb/gen/BUILD.bazel for more details. # # gazelle:resolve proto go roachpb/api.proto //pkg/roachpb -# gazelle:resolve proto go roachpb/app_stats.proto //pkg/roachpb # gazelle:resolve proto go roachpb/data.proto //pkg/roachpb # gazelle:resolve proto go roachpb/errors.proto //pkg/roachpb # gazelle:resolve proto go roachpb/index_usage_stats.proto //pkg/roachpb diff --git a/docs/generated/http/BUILD.bazel b/docs/generated/http/BUILD.bazel index 1fab1d15ef41..2ad3dbaf2630 100644 --- a/docs/generated/http/BUILD.bazel +++ b/docs/generated/http/BUILD.bazel @@ -20,6 +20,7 @@ genrule( "//pkg/server/serverpb:serverpb_proto", "//pkg/server/status/statuspb:statuspb_proto", "//pkg/settings:settings_proto", + "//pkg/sql/appstatspb:appstatspb_proto", "//pkg/sql/catalog/catenumpb:catenumpb_proto", "//pkg/sql/catalog/catpb:catpb_proto", "//pkg/sql/catalog/descpb:descpb_proto", diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index cd172954169a..12f3f5a85bc7 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -323,11 +323,11 @@ Immutable cardinality(input: anyelement[]) → int

Calculates the number of elements contained in input

Immutable -crdb_internal.merge_statement_stats(input: jsonb[]) → jsonb

Merge an array of roachpb.StatementStatistics into a single JSONB object

+crdb_internal.merge_statement_stats(input: jsonb[]) → jsonb

Merge an array of appstatspb.StatementStatistics into a single JSONB object

Immutable crdb_internal.merge_stats_metadata(input: jsonb[]) → jsonb

Merge an array of StmtStatsMetadata into a single JSONB object

Immutable -crdb_internal.merge_transaction_stats(input: jsonb[]) → jsonb

Merge an array of roachpb.TransactionStatistics into a single JSONB object

+crdb_internal.merge_transaction_stats(input: jsonb[]) → jsonb

Merge an array of appstatspb.TransactionStatistics into a single JSONB object

Immutable string_to_array(str: string, delimiter: string) → string[]

Split a string into components on a delimiter.

Immutable diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index f4b8f4df8e0e..031020781546 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -282,6 +282,7 @@ ALL_TESTS = [ "//pkg/spanconfig/spanconfigstore:spanconfigstore_test", "//pkg/spanconfig/spanconfigtestutils:spanconfigtestutils_test", "//pkg/spanconfig:spanconfig_test", + "//pkg/sql/appstatspb:appstatspb_test", "//pkg/sql/backfill:backfill_test", "//pkg/sql/cacheutil:cacheutil_test", "//pkg/sql/catalog/bootstrap:bootstrap_test", @@ -1429,6 +1430,8 @@ GO_TARGETS = [ "//pkg/spanconfig/spanconfigtestutils:spanconfigtestutils_test", "//pkg/spanconfig:spanconfig", "//pkg/spanconfig:spanconfig_test", + "//pkg/sql/appstatspb:appstatspb", + "//pkg/sql/appstatspb:appstatspb_test", "//pkg/sql/backfill:backfill", "//pkg/sql/backfill:backfill_test", "//pkg/sql/cacheutil:cacheutil", @@ -2744,6 +2747,7 @@ GET_X_DATA_TARGETS = [ "//pkg/spanconfig/spanconfigtestutils:get_x_data", "//pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster:get_x_data", "//pkg/sql:get_x_data", + "//pkg/sql/appstatspb:get_x_data", "//pkg/sql/backfill:get_x_data", "//pkg/sql/cacheutil:get_x_data", "//pkg/sql/catalog:get_x_data", diff --git a/pkg/ccl/serverccl/statusccl/BUILD.bazel b/pkg/ccl/serverccl/statusccl/BUILD.bazel index d69c5c8cb034..34ed9f9cbfe2 100644 --- a/pkg/ccl/serverccl/statusccl/BUILD.bazel +++ b/pkg/ccl/serverccl/statusccl/BUILD.bazel @@ -25,6 +25,7 @@ go_test( "//pkg/server", "//pkg/server/serverpb", "//pkg/spanconfig", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/descpb", "//pkg/sql/clusterunique", "//pkg/sql/sem/catconstants", diff --git a/pkg/ccl/serverccl/statusccl/tenant_status_test.go b/pkg/ccl/serverccl/statusccl/tenant_status_test.go index 5859b8b4015a..ba35f0025226 100644 --- a/pkg/ccl/serverccl/statusccl/tenant_status_test.go +++ b/pkg/ccl/serverccl/statusccl/tenant_status_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" @@ -1136,7 +1137,7 @@ func testTxnIDResolutionRPC(ctx context.Context, t *testing.T, helper serverccl. require.Equal(t, txnID, resp.ResolvedTxnIDs[0].TxnID, "expected to find txn %s on coordinator node %d, but it "+ "was not", txnID.String(), coordinatorNodeID) - require.NotEqual(t, roachpb.InvalidTransactionFingerprintID, resp.ResolvedTxnIDs[0].TxnFingerprintID) + require.NotEqual(t, appstatspb.InvalidTransactionFingerprintID, resp.ResolvedTxnIDs[0].TxnFingerprintID) return nil }) } diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index fdeda998c619..99dd0336bcc0 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -53,6 +53,7 @@ PROTOBUF_SRCS = [ "//pkg/server/serverpb:serverpb_go_proto", "//pkg/server/status/statuspb:statuspb_go_proto", "//pkg/settings:settings_go_proto", + "//pkg/sql/appstatspb:appstatspb_go_proto", "//pkg/sql/catalog/catenumpb:catenumpb_go_proto", "//pkg/sql/catalog/catpb:catpb_go_proto", "//pkg/sql/catalog/descpb:descpb_go_proto", diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index e9e9b53acd46..286159f39236 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -12,7 +12,6 @@ go_library( "ambiguous_result_error.go", "api.go", "api_requestheader.go", - "app_stats.go", "batch.go", "data.go", "errors.go", @@ -73,7 +72,6 @@ go_test( size = "small", srcs = [ "api_test.go", - "app_stats_test.go", "batch_test.go", "data_test.go", "errors_test.go", @@ -135,7 +133,6 @@ proto_library( name = "roachpb_proto", srcs = [ "api.proto", - "app_stats.proto", "data.proto", "errors.proto", "index_usage_stats.proto", diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 28551827f99e..889536b47f86 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -180,6 +180,7 @@ go_library( "//pkg/spanconfig/spanconfigsqltranslator", "//pkg/spanconfig/spanconfigsqlwatcher", "//pkg/sql", + "//pkg/sql/appstatspb", "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", @@ -464,6 +465,7 @@ go_test( "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/sql", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/descpb", "//pkg/sql/clusterunique", "//pkg/sql/execinfrapb", diff --git a/pkg/server/combined_statement_stats.go b/pkg/server/combined_statement_stats.go index 0f3513a29fee..74359fa4dd77 100644 --- a/pkg/server/combined_statement_stats.go +++ b/pkg/server/combined_statement_stats.go @@ -18,10 +18,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -199,7 +199,7 @@ func collectCombinedStatements( app := string(tree.MustBeDString(row[2])) aggregatedTs := tree.MustBeDTimestampTZ(row[3]).Time - var metadata roachpb.CollectedStatementStatistics + var metadata appstatspb.CollectedStatementStatistics metadataJSON := tree.MustBeDJSON(row[4]).JSON if err = sqlstatsutil.DecodeStmtStatsMetadataJSON(metadataJSON, &metadata); err != nil { return nil, serverError(ctx, err) @@ -207,7 +207,7 @@ func collectCombinedStatements( metadata.Key.App = app metadata.Key.TransactionFingerprintID = - roachpb.TransactionFingerprintID(transactionFingerprintID) + appstatspb.TransactionFingerprintID(transactionFingerprintID) statsJSON := tree.MustBeDJSON(row[5]).JSON if err = sqlstatsutil.DecodeStmtStatsStatisticsJSON(statsJSON, &metadata.Stats); err != nil { @@ -229,7 +229,7 @@ func collectCombinedStatements( AggregatedTs: aggregatedTs, AggregationInterval: time.Duration(aggInterval.Nanos()), }, - ID: roachpb.StmtFingerprintID(statementFingerprintID), + ID: appstatspb.StmtFingerprintID(statementFingerprintID), Stats: metadata.Stats, } @@ -303,7 +303,7 @@ func collectCombinedTransactions( return nil, serverError(ctx, err) } - var metadata roachpb.CollectedTransactionStatistics + var metadata appstatspb.CollectedTransactionStatistics metadataJSON := tree.MustBeDJSON(row[3]).JSON if err = sqlstatsutil.DecodeTxnStatsMetadataJSON(metadataJSON, &metadata); err != nil { return nil, serverError(ctx, err) @@ -317,13 +317,13 @@ func collectCombinedTransactions( aggInterval := tree.MustBeDInterval(row[5]).Duration txnStats := serverpb.StatementsResponse_ExtendedCollectedTransactionStatistics{ - StatsData: roachpb.CollectedTransactionStatistics{ + StatsData: appstatspb.CollectedTransactionStatistics{ StatementFingerprintIDs: metadata.StatementFingerprintIDs, App: app, Stats: metadata.Stats, AggregatedTs: aggregatedTs, AggregationInterval: time.Duration(aggInterval.Nanos()), - TransactionFingerprintID: roachpb.TransactionFingerprintID(fingerprintID), + TransactionFingerprintID: appstatspb.TransactionFingerprintID(fingerprintID), }, } @@ -509,8 +509,8 @@ func getTotalStatementDetails( return statement, serverError(ctx, errors.Newf("expected %d columns, received %d", expectedNumDatums)) } - var statistics roachpb.CollectedStatementStatistics - var aggregatedMetadata roachpb.AggregatedStatementMetadata + var statistics appstatspb.CollectedStatementStatistics + var aggregatedMetadata appstatspb.AggregatedStatementMetadata metadataJSON := tree.MustBeDJSON(row[0]).JSON if err = sqlstatsutil.DecodeAggregatedMetadataJSON(metadataJSON, &aggregatedMetadata); err != nil { @@ -613,8 +613,8 @@ func getStatementDetailsPerAggregatedTs( aggregatedTs := tree.MustBeDTimestampTZ(row[0]).Time - var metadata roachpb.CollectedStatementStatistics - var aggregatedMetadata roachpb.AggregatedStatementMetadata + var metadata appstatspb.CollectedStatementStatistics + var aggregatedMetadata appstatspb.AggregatedStatementMetadata metadataJSON := tree.MustBeDJSON(row[1]).JSON if err = sqlstatsutil.DecodeAggregatedMetadataJSON(metadataJSON, &aggregatedMetadata); err != nil { return nil, serverError(ctx, err) @@ -797,8 +797,8 @@ func getStatementDetailsPerPlanHash( explainPlan = getExplainPlanFromGist(ctx, ie, planGist) } - var metadata roachpb.CollectedStatementStatistics - var aggregatedMetadata roachpb.AggregatedStatementMetadata + var metadata appstatspb.CollectedStatementStatistics + var aggregatedMetadata appstatspb.AggregatedStatementMetadata metadataJSON := tree.MustBeDJSON(row[2]).JSON if err = sqlstatsutil.DecodeAggregatedMetadataJSON(metadataJSON, &aggregatedMetadata); err != nil { return nil, serverError(ctx, err) diff --git a/pkg/server/diagnostics/diagnosticspb/BUILD.bazel b/pkg/server/diagnostics/diagnosticspb/BUILD.bazel index 50a4b598ee18..b675c587bccc 100644 --- a/pkg/server/diagnostics/diagnosticspb/BUILD.bazel +++ b/pkg/server/diagnostics/diagnosticspb/BUILD.bazel @@ -12,6 +12,7 @@ proto_library( "//pkg/build:build_proto", "//pkg/config/zonepb:zonepb_proto", "//pkg/roachpb:roachpb_proto", + "//pkg/sql/appstatspb:appstatspb_proto", "//pkg/sql/catalog/descpb:descpb_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", ], @@ -28,6 +29,7 @@ go_proto_library( "//pkg/build", "//pkg/config/zonepb", "//pkg/roachpb", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/descpb", "@com_github_gogo_protobuf//gogoproto", ], diff --git a/pkg/server/diagnostics/diagnosticspb/diagnostics.proto b/pkg/server/diagnostics/diagnosticspb/diagnostics.proto index d630f76992fa..0ffc70d815f8 100644 --- a/pkg/server/diagnostics/diagnosticspb/diagnostics.proto +++ b/pkg/server/diagnostics/diagnosticspb/diagnostics.proto @@ -15,7 +15,7 @@ option go_package = "diagnosticspb"; import "build/info.proto"; import "config/zonepb/zone.proto"; import "gogoproto/gogo.proto"; -import "roachpb/app_stats.proto"; +import "sql/appstatspb/app_stats.proto"; import "sql/catalog/descpb/structured.proto"; import "roachpb/metadata.proto"; diff --git a/pkg/server/drain_test.go b/pkg/server/drain_test.go index 27bc8b02e1dc..d9fced71ed94 100644 --- a/pkg/server/drain_test.go +++ b/pkg/server/drain_test.go @@ -17,10 +17,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -127,7 +127,7 @@ INSERT INTO t.test VALUES (3); stats, err := ts.GetScrubbedStmtStats(ctx) require.NoError(t, err) require.Truef(t, - func(stats []roachpb.CollectedStatementStatistics) bool { + func(stats []appstatspb.CollectedStatementStatistics) bool { for _, stat := range stats { if stat.Key.Query == "INSERT INTO _ VALUES (_)" { return true diff --git a/pkg/server/serverpb/BUILD.bazel b/pkg/server/serverpb/BUILD.bazel index 9a50a25685bc..460f1d909c47 100644 --- a/pkg/server/serverpb/BUILD.bazel +++ b/pkg/server/serverpb/BUILD.bazel @@ -29,6 +29,7 @@ proto_library( "//pkg/roachpb:roachpb_proto", "//pkg/server/diagnostics/diagnosticspb:diagnosticspb_proto", "//pkg/server/status/statuspb:statuspb_proto", + "//pkg/sql/appstatspb:appstatspb_proto", "//pkg/sql/contentionpb:contentionpb_proto", "//pkg/sql/sqlstats/insights:insights_proto", "//pkg/storage/enginepb:enginepb_proto", @@ -68,6 +69,7 @@ go_proto_library( "//pkg/roachpb", "//pkg/server/diagnostics/diagnosticspb", "//pkg/server/status/statuspb", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/descpb", # keep "//pkg/sql/contentionpb", "//pkg/sql/execinfrapb", # keep diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index 7d26c0e4f6e9..fb59daef508b 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -16,7 +16,6 @@ import "build/info.proto"; import "errorspb/errors.proto"; import "gossip/gossip.proto"; import "jobs/jobspb/jobs.proto"; -import "roachpb/app_stats.proto"; import "roachpb/data.proto"; import "roachpb/index_usage_stats.proto"; import "roachpb/span_config.proto"; @@ -24,6 +23,7 @@ import "roachpb/metadata.proto"; import "server/diagnostics/diagnosticspb/diagnostics.proto"; import "server/serverpb/index_recommendations.proto"; import "server/status/statuspb/status.proto"; +import "sql/appstatspb/app_stats.proto"; import "sql/contentionpb/contention.proto"; import "sql/sqlstats/insights/insights.proto"; import "storage/enginepb/engine.proto"; @@ -1001,7 +1001,7 @@ message Session { // List of transaction fingerprint IDs in this session. repeated uint64 txn_fingerprint_ids = 17 [(gogoproto.customname) = "TxnFingerprintIDs", - (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TransactionFingerprintID", + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.TransactionFingerprintID", (gogoproto.nullable) = false]; // The session's total active time. @@ -1608,7 +1608,7 @@ message StatementsResponse { message CollectedStatementStatistics { ExtendedStatementStatisticsKey key = 1 [(gogoproto.nullable) = false]; uint64 id = 3 [(gogoproto.customname) = "ID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtFingerprintID"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.StmtFingerprintID"]; cockroach.sql.StatementStatistics stats = 2 [(gogoproto.nullable) = false]; } diff --git a/pkg/server/statements.go b/pkg/server/statements.go index c34268cc76b5..cfa55475b0b2 100644 --- a/pkg/server/statements.go +++ b/pkg/server/statements.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "google.golang.org/grpc/codes" @@ -109,8 +110,8 @@ func statementsLocal( sqlServer *SQLServer, fetchMode serverpb.StatementsRequest_FetchMode, ) (*serverpb.StatementsResponse, error) { - var stmtStats []roachpb.CollectedStatementStatistics - var txnStats []roachpb.CollectedTransactionStatistics + var stmtStats []appstatspb.CollectedStatementStatistics + var txnStats []appstatspb.CollectedTransactionStatistics var err error if fetchMode != serverpb.StatementsRequest_TxnStatsOnly { diff --git a/pkg/server/stats_test.go b/pkg/server/stats_test.go index bedca43947fb..9a95ff1b250d 100644 --- a/pkg/server/stats_test.go +++ b/pkg/server/stats_test.go @@ -18,11 +18,11 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/diagnostics" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -202,7 +202,7 @@ func TestSQLStatCollection(t *testing.T) { require.NoError(t, err) foundStat := false - var sqlStatData roachpb.StatementStatistics + var sqlStatData appstatspb.StatementStatistics for _, stat := range stats { if stat.Key.Query == "INSERT INTO _ VALUES (_)" { diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index 84006153dc6e..6d6ad4ea9b5b 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -1644,7 +1645,7 @@ func TestStatusAPICombinedTransactions(t *testing.T) { } // Construct a map of all the statement fingerprint IDs. - statementFingerprintIDs := make(map[roachpb.StmtFingerprintID]bool, len(resp.Statements)) + statementFingerprintIDs := make(map[appstatspb.StmtFingerprintID]bool, len(resp.Statements)) for _, respStatement := range resp.Statements { statementFingerprintIDs[respStatement.ID] = true } @@ -1779,7 +1780,7 @@ func TestStatusAPITransactions(t *testing.T) { } // Construct a map of all the statement fingerprint IDs. - statementFingerprintIDs := make(map[roachpb.StmtFingerprintID]bool, len(resp.Statements)) + statementFingerprintIDs := make(map[appstatspb.StmtFingerprintID]bool, len(resp.Statements)) for _, respStatement := range resp.Statements { statementFingerprintIDs[respStatement.ID] = true } @@ -2168,7 +2169,7 @@ func TestStatusAPIStatementDetails(t *testing.T) { thirdServerSQL.Exec(t, stmt) } query := `INSERT INTO posts VALUES (_, '_')` - fingerprintID := roachpb.ConstructStatementFingerprintID(query, + fingerprintID := appstatspb.ConstructStatementFingerprintID(query, false, true, `roachblog`) path := fmt.Sprintf(`stmtdetails/%v`, fingerprintID) @@ -2362,7 +2363,7 @@ func TestStatusAPIStatementDetails(t *testing.T) { } selectQuery := "SELECT _, _, _, _" - fingerprintID = roachpb.ConstructStatementFingerprintID(selectQuery, false, + fingerprintID = appstatspb.ConstructStatementFingerprintID(selectQuery, false, true, "defaultdb") testPath( diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 123eb22bf6bd..aabf1b77e3fc 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -338,6 +338,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/sql/appstatspb", "//pkg/sql/backfill", "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", @@ -723,6 +724,7 @@ go_test( "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/spanconfig/spanconfigptsreader", + "//pkg/sql/appstatspb", "//pkg/sql/backfill", "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", diff --git a/pkg/sql/appstatspb/BUILD.bazel b/pkg/sql/appstatspb/BUILD.bazel new file mode 100644 index 000000000000..f286ad52d628 --- /dev/null +++ b/pkg/sql/appstatspb/BUILD.bazel @@ -0,0 +1,44 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "appstatspb", + srcs = ["app_stats.go"], + embed = [":appstatspb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb", + visibility = ["//visibility:public"], + deps = ["//pkg/util"], +) + +go_test( + name = "appstatspb_test", + srcs = ["app_stats_test.go"], + args = ["-test.timeout=295s"], + embed = [":appstatspb"], + deps = ["@com_github_stretchr_testify//require"], +) + +proto_library( + name = "appstatspb_proto", + srcs = ["app_stats.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = [ + "@com_github_gogo_protobuf//gogoproto:gogo_proto", + "@com_google_protobuf//:duration_proto", + "@com_google_protobuf//:timestamp_proto", + ], +) + +go_proto_library( + name = "appstatspb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb", + proto = ":appstatspb_proto", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto"], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/roachpb/app_stats.go b/pkg/sql/appstatspb/app_stats.go similarity index 99% rename from pkg/roachpb/app_stats.go rename to pkg/sql/appstatspb/app_stats.go index fa47373c5c87..6c9a277d80b4 100644 --- a/pkg/roachpb/app_stats.go +++ b/pkg/sql/appstatspb/app_stats.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package appstatspb import ( "math" diff --git a/pkg/roachpb/app_stats.proto b/pkg/sql/appstatspb/app_stats.proto similarity index 99% rename from pkg/roachpb/app_stats.proto rename to pkg/sql/appstatspb/app_stats.proto index 5f6c244b2d94..d83cac1691c6 100644 --- a/pkg/roachpb/app_stats.proto +++ b/pkg/sql/appstatspb/app_stats.proto @@ -10,7 +10,7 @@ syntax = "proto2"; package cockroach.sql; -option go_package = "roachpb"; +option go_package = "appstatspb"; import "gogoproto/gogo.proto"; import "google/protobuf/timestamp.proto"; diff --git a/pkg/roachpb/app_stats_test.go b/pkg/sql/appstatspb/app_stats_test.go similarity index 99% rename from pkg/roachpb/app_stats_test.go rename to pkg/sql/appstatspb/app_stats_test.go index bef6ea52971e..a7e50a8b27a7 100644 --- a/pkg/roachpb/app_stats_test.go +++ b/pkg/sql/appstatspb/app_stats_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package appstatspb import ( "math" diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 8cc4ed2c0350..83a455664a0b 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catsessiondata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen" @@ -585,7 +586,7 @@ func (s *Server) GetTxnIDCache() *txnidcache.Cache { // scrubbed will be omitted from the returned map. func (s *Server) GetScrubbedStmtStats( ctx context.Context, -) ([]roachpb.CollectedStatementStatistics, error) { +) ([]appstatspb.CollectedStatementStatistics, error) { return s.getScrubbedStmtStats(ctx, s.sqlStats.GetLocalMemProvider()) } @@ -596,9 +597,9 @@ var _ = (*Server).GetScrubbedStmtStats // identifiers (e.g. table and column names) aren't scrubbed from the statements. func (s *Server) GetUnscrubbedStmtStats( ctx context.Context, -) ([]roachpb.CollectedStatementStatistics, error) { - var stmtStats []roachpb.CollectedStatementStatistics - stmtStatsVisitor := func(_ context.Context, stat *roachpb.CollectedStatementStatistics) error { +) ([]appstatspb.CollectedStatementStatistics, error) { + var stmtStats []appstatspb.CollectedStatementStatistics + stmtStatsVisitor := func(_ context.Context, stat *appstatspb.CollectedStatementStatistics) error { stmtStats = append(stmtStats, *stat) return nil } @@ -616,9 +617,9 @@ func (s *Server) GetUnscrubbedStmtStats( // Identifiers (e.g. table and column names) aren't scrubbed from the statements. func (s *Server) GetUnscrubbedTxnStats( ctx context.Context, -) ([]roachpb.CollectedTransactionStatistics, error) { - var txnStats []roachpb.CollectedTransactionStatistics - txnStatsVisitor := func(_ context.Context, stat *roachpb.CollectedTransactionStatistics) error { +) ([]appstatspb.CollectedTransactionStatistics, error) { + var txnStats []appstatspb.CollectedTransactionStatistics + txnStatsVisitor := func(_ context.Context, stat *appstatspb.CollectedTransactionStatistics) error { txnStats = append(txnStats, *stat) return nil } @@ -636,17 +637,17 @@ func (s *Server) GetUnscrubbedTxnStats( // returns statistics from the reported stats pool. func (s *Server) GetScrubbedReportingStats( ctx context.Context, -) ([]roachpb.CollectedStatementStatistics, error) { +) ([]appstatspb.CollectedStatementStatistics, error) { return s.getScrubbedStmtStats(ctx, s.reportedStats) } func (s *Server) getScrubbedStmtStats( ctx context.Context, statsProvider sqlstats.Provider, -) ([]roachpb.CollectedStatementStatistics, error) { +) ([]appstatspb.CollectedStatementStatistics, error) { salt := ClusterSecret.Get(&s.cfg.Settings.SV) - var scrubbedStats []roachpb.CollectedStatementStatistics - stmtStatsVisitor := func(_ context.Context, stat *roachpb.CollectedStatementStatistics) error { + var scrubbedStats []appstatspb.CollectedStatementStatistics + stmtStatsVisitor := func(_ context.Context, stat *appstatspb.CollectedStatementStatistics) error { // Scrub the statement itself. scrubbedQueryStr, ok := scrubStmtStatKey(s.cfg.VirtualSchemas, stat.Key.Query) @@ -1372,7 +1373,7 @@ type connExecutor struct { // transactionStatementFingerprintIDs tracks all statement IDs that make up the current // transaction. It's length is bound by the TxnStatsNumStmtFingerprintIDsToRecord // cluster setting. - transactionStatementFingerprintIDs []roachpb.StmtFingerprintID + transactionStatementFingerprintIDs []appstatspb.StmtFingerprintID // transactionStatementsHash is the hashed accumulation of all statementFingerprintIDs // that comprise the transaction. It is used to construct the key when @@ -2489,7 +2490,7 @@ func (ex *connExecutor) execCopyIn( // These fields are not available in COPY, so use the empty value. f := tree.NewFmtCtx(tree.FmtHideConstants) f.FormatNode(cmd.Stmt) - stmtFingerprintID := roachpb.ConstructStatementFingerprintID( + stmtFingerprintID := appstatspb.ConstructStatementFingerprintID( f.CloseAndGetString(), copyErr != nil, ex.implicitTxn(), diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 3cd46ad7ab23..cdec8677299d 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" @@ -1156,7 +1157,7 @@ func (ex *connExecutor) dispatchToExecutionEngine( res.DisableBuffering() } - var stmtFingerprintID roachpb.StmtFingerprintID + var stmtFingerprintID appstatspb.StmtFingerprintID var stats topLevelQueryStats defer func() { planner.maybeLogStatement( @@ -2296,7 +2297,7 @@ func (ex *connExecutor) onTxnFinish(ctx context.Context, ev txnEvent) { implicit := ex.extraTxnState.txnFinishClosure.implicit ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionEndExecTransaction, timeutil.Now()) transactionFingerprintID := - roachpb.TransactionFingerprintID(ex.extraTxnState.transactionStatementsHash.Sum()) + appstatspb.TransactionFingerprintID(ex.extraTxnState.transactionStatementsHash.Sum()) err := ex.txnFingerprintIDCache.Add(transactionFingerprintID) if err != nil { @@ -2355,7 +2356,7 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) { // execution. ex.txnIDCacheWriter.Record(contentionpb.ResolvedTxnID{ TxnID: txnID, - TxnFingerprintID: roachpb.InvalidTransactionFingerprintID, + TxnFingerprintID: appstatspb.InvalidTransactionFingerprintID, }) ex.state.mu.RLock() @@ -2397,7 +2398,7 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) { func (ex *connExecutor) recordTransactionFinish( ctx context.Context, - transactionFingerprintID roachpb.TransactionFingerprintID, + transactionFingerprintID appstatspb.TransactionFingerprintID, ev txnEvent, implicit bool, txnStart time.Time, diff --git a/pkg/sql/contention/BUILD.bazel b/pkg/sql/contention/BUILD.bazel index 3a22a13ba22e..e1ff777162b9 100644 --- a/pkg/sql/contention/BUILD.bazel +++ b/pkg/sql/contention/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/server/serverpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/descpb", "//pkg/sql/contention/contentionutils", "//pkg/sql/contentionpb", @@ -50,6 +51,7 @@ go_test( "//pkg/keys", "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/contentionpb", "//pkg/storage/enginepb", "//pkg/testutils", diff --git a/pkg/sql/contention/event_store_test.go b/pkg/sql/contention/event_store_test.go index 2bce498cfd49..6a20a6c4f3b7 100644 --- a/pkg/sql/contention/event_store_test.go +++ b/pkg/sql/contention/event_store_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -89,11 +90,11 @@ func TestEventStore(t *testing.T) { "be at least %s, but it is %s", expectedEvent.CollectionTs.String(), actual.CollectionTs.String()) } - if actual.BlockingTxnFingerprintID != roachpb.InvalidTransactionFingerprintID { + if actual.BlockingTxnFingerprintID != appstatspb.InvalidTransactionFingerprintID { return errors.Newf("expect blocking txn fingerprint id to be invalid, "+ "but it is %d", actual.BlockingTxnFingerprintID) } - if actual.WaitingTxnFingerprintID != roachpb.InvalidTransactionFingerprintID { + if actual.WaitingTxnFingerprintID != appstatspb.InvalidTransactionFingerprintID { return errors.Newf("expect waiting txn fingerprint id to be invalid, "+ "but it is %d", actual.WaitingTxnFingerprintID) } @@ -271,11 +272,11 @@ func randomlyGenerateTestData(testSize int, numOfCoordinator int) []testData { tcs = append(tcs, testData{ blockingTxn: contentionpb.ResolvedTxnID{ TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.TransactionFingerprintID(math.MaxUint64 - uint64(i)), + TxnFingerprintID: appstatspb.TransactionFingerprintID(math.MaxUint64 - uint64(i)), }, waitingTxn: contentionpb.ResolvedTxnID{ TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.TransactionFingerprintID(math.MaxUint64/2 - uint64(i)), + TxnFingerprintID: appstatspb.TransactionFingerprintID(math.MaxUint64/2 - uint64(i)), }, coordinatorNodeID: strconv.Itoa(rand.Intn(numOfCoordinator)), }) diff --git a/pkg/sql/contention/resolver.go b/pkg/sql/contention/resolver.go index 1cf51cac28dc..c1310ab5fed4 100644 --- a/pkg/sql/contention/resolver.go +++ b/pkg/sql/contention/resolver.go @@ -16,8 +16,8 @@ import ( "sort" "strconv" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -233,12 +233,12 @@ func (q *resolverQueueImpl) resolveLocked(ctx context.Context) error { func maybeUpdateTxnFingerprintID( txnID uuid.UUID, - existingTxnFingerprintID *roachpb.TransactionFingerprintID, - resolvedTxnIDs, inProgressTxnIDs map[uuid.UUID]roachpb.TransactionFingerprintID, + existingTxnFingerprintID *appstatspb.TransactionFingerprintID, + resolvedTxnIDs, inProgressTxnIDs map[uuid.UUID]appstatspb.TransactionFingerprintID, ) (needToRetry bool, initialRetryBudget uint32) { // This means the txnID has already been resolved into transaction fingerprint // ID. - if *existingTxnFingerprintID != roachpb.InvalidTransactionFingerprintID { + if *existingTxnFingerprintID != appstatspb.InvalidTransactionFingerprintID { return false /* needToRetry */, 0 /* initialRetryBudget */ } @@ -324,17 +324,17 @@ func readUntilNextCoordinatorID( func extractResolvedAndInProgressTxnIDs( resp *serverpb.TxnIDResolutionResponse, -) (resolvedTxnIDs, inProgressTxnIDs map[uuid.UUID]roachpb.TransactionFingerprintID) { +) (resolvedTxnIDs, inProgressTxnIDs map[uuid.UUID]appstatspb.TransactionFingerprintID) { if resp == nil { return nil /* resolvedTxnID */, nil /* inProgressTxnIDs */ } - resolvedTxnIDs = make(map[uuid.UUID]roachpb.TransactionFingerprintID, len(resp.ResolvedTxnIDs)) - inProgressTxnIDs = make(map[uuid.UUID]roachpb.TransactionFingerprintID, len(resp.ResolvedTxnIDs)) + resolvedTxnIDs = make(map[uuid.UUID]appstatspb.TransactionFingerprintID, len(resp.ResolvedTxnIDs)) + inProgressTxnIDs = make(map[uuid.UUID]appstatspb.TransactionFingerprintID, len(resp.ResolvedTxnIDs)) for _, event := range resp.ResolvedTxnIDs { - if event.TxnFingerprintID == roachpb.InvalidTransactionFingerprintID { - inProgressTxnIDs[event.TxnID] = roachpb.InvalidTransactionFingerprintID + if event.TxnFingerprintID == appstatspb.InvalidTransactionFingerprintID { + inProgressTxnIDs[event.TxnID] = appstatspb.InvalidTransactionFingerprintID } else { resolvedTxnIDs[event.TxnID] = event.TxnFingerprintID } @@ -360,10 +360,10 @@ func makeRPCRequestsFromBatch( } for i := range batch { - if batch[i].BlockingTxnFingerprintID == roachpb.InvalidTransactionFingerprintID { + if batch[i].BlockingTxnFingerprintID == appstatspb.InvalidTransactionFingerprintID { blockingTxnIDReq.TxnIDs = append(blockingTxnIDReq.TxnIDs, batch[i].BlockingEvent.TxnMeta.ID) } - if batch[i].WaitingTxnFingerprintID == roachpb.InvalidTransactionFingerprintID { + if batch[i].WaitingTxnFingerprintID == appstatspb.InvalidTransactionFingerprintID { waitingTxnIDReq.TxnIDs = append(waitingTxnIDReq.TxnIDs, batch[i].WaitingTxnID) } } diff --git a/pkg/sql/contention/resolver_test.go b/pkg/sql/contention/resolver_test.go index 9505cde6b387..c246677494e1 100644 --- a/pkg/sql/contention/resolver_test.go +++ b/pkg/sql/contention/resolver_test.go @@ -18,7 +18,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -142,7 +142,7 @@ func TestResolver(t *testing.T) { { blockingTxn: contentionpb.ResolvedTxnID{ TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.InvalidTransactionFingerprintID, + TxnFingerprintID: appstatspb.InvalidTransactionFingerprintID, }, waitingTxn: contentionpb.ResolvedTxnID{ TxnID: uuid.FastMakeV4(), @@ -168,18 +168,18 @@ func TestResolver(t *testing.T) { }, waitingTxn: contentionpb.ResolvedTxnID{ TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.InvalidTransactionFingerprintID, + TxnFingerprintID: appstatspb.InvalidTransactionFingerprintID, }, coordinatorNodeID: "2", }, { blockingTxn: contentionpb.ResolvedTxnID{ TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.InvalidTransactionFingerprintID, + TxnFingerprintID: appstatspb.InvalidTransactionFingerprintID, }, waitingTxn: contentionpb.ResolvedTxnID{ TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.InvalidTransactionFingerprintID, + TxnFingerprintID: appstatspb.InvalidTransactionFingerprintID, }, coordinatorNodeID: "2", }, @@ -275,7 +275,7 @@ func TestResolver(t *testing.T) { // Even we are retrying without creating retry budget, we should still update // the fields on the contention event. require.Equal(t, - roachpb.InvalidTransactionFingerprintID, + appstatspb.InvalidTransactionFingerprintID, resolver.mu.unresolvedEvents[1].WaitingTxnFingerprintID) require.Equal(t, inProgressContentionEventDueToInProgressBlockingAndWaitingTxn.blockingTxn.TxnFingerprintID, @@ -554,8 +554,8 @@ func generateUnresolvedContentionEventsFromTestData( inputEvent.WaitingTxnID = tc.waitingTxn.TxnID input = append(input, inputEvent) - if tc.blockingTxn.TxnFingerprintID != roachpb.InvalidTransactionFingerprintID && - tc.waitingTxn.TxnFingerprintID != roachpb.InvalidTransactionFingerprintID { + if tc.blockingTxn.TxnFingerprintID != appstatspb.InvalidTransactionFingerprintID && + tc.waitingTxn.TxnFingerprintID != appstatspb.InvalidTransactionFingerprintID { expectedResolvedEvent := contentionpb.ExtendedContentionEvent{} expectedResolvedEvent.BlockingEvent = inputEvent.BlockingEvent diff --git a/pkg/sql/contention/test_utils.go b/pkg/sql/contention/test_utils.go index 294625d3f8e0..0f4b6e944e79 100644 --- a/pkg/sql/contention/test_utils.go +++ b/pkg/sql/contention/test_utils.go @@ -13,20 +13,20 @@ package contention import ( "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) type fakeStatusServer struct { - data map[uuid.UUID]roachpb.TransactionFingerprintID + data map[uuid.UUID]appstatspb.TransactionFingerprintID injectedError error } func newFakeStatusServer() *fakeStatusServer { return &fakeStatusServer{ - data: make(map[uuid.UUID]roachpb.TransactionFingerprintID), + data: make(map[uuid.UUID]appstatspb.TransactionFingerprintID), injectedError: nil, } } @@ -55,7 +55,7 @@ func (f *fakeStatusServer) txnIDResolution( } func (f *fakeStatusServer) setTxnIDEntry( - txnID uuid.UUID, txnFingerprintID roachpb.TransactionFingerprintID, + txnID uuid.UUID, txnFingerprintID appstatspb.TransactionFingerprintID, ) { f.data[txnID] = txnFingerprintID } @@ -82,7 +82,7 @@ func (f fakeStatusServerCluster) txnIDResolution( } func (f fakeStatusServerCluster) setTxnIDEntry( - coordinatorNodeID string, txnID uuid.UUID, txnFingerprintID roachpb.TransactionFingerprintID, + coordinatorNodeID string, txnID uuid.UUID, txnFingerprintID appstatspb.TransactionFingerprintID, ) { f.getStatusServer(coordinatorNodeID).setTxnIDEntry(txnID, txnFingerprintID) } diff --git a/pkg/sql/contention/txnidcache/BUILD.bazel b/pkg/sql/contention/txnidcache/BUILD.bazel index 4661329f1422..3aacc3167e2b 100644 --- a/pkg/sql/contention/txnidcache/BUILD.bazel +++ b/pkg/sql/contention/txnidcache/BUILD.bazel @@ -14,9 +14,9 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/contention/txnidcache", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/contention/contentionutils", "//pkg/sql/contentionpb", "//pkg/util/encoding", @@ -40,12 +40,12 @@ go_test( embed = [":txnidcache"], deps = [ "//pkg/kv", - "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", "//pkg/settings/cluster", "//pkg/sql", + "//pkg/sql/appstatspb", "//pkg/sql/contentionpb", "//pkg/sql/sessiondata", "//pkg/sql/tests", diff --git a/pkg/sql/contention/txnidcache/fifo_cache.go b/pkg/sql/contention/txnidcache/fifo_cache.go index aef77400207e..c02ffb9742e3 100644 --- a/pkg/sql/contention/txnidcache/fifo_cache.go +++ b/pkg/sql/contention/txnidcache/fifo_cache.go @@ -14,7 +14,7 @@ import ( "sync" "unsafe" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contention/contentionutils" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -32,7 +32,7 @@ type fifoCache struct { mu struct { syncutil.RWMutex - data map[uuid.UUID]roachpb.TransactionFingerprintID + data map[uuid.UUID]appstatspb.TransactionFingerprintID eviction blockList } @@ -57,7 +57,7 @@ func newFIFOCache(capacity contentionutils.CapacityLimiter) *fifoCache { capacity: capacity, } - c.mu.data = make(map[uuid.UUID]roachpb.TransactionFingerprintID) + c.mu.data = make(map[uuid.UUID]appstatspb.TransactionFingerprintID) c.mu.eviction = blockList{} return c } @@ -79,7 +79,7 @@ func (c *fifoCache) add(b *block) { c.maybeEvictLocked() } -func (c *fifoCache) get(txnID uuid.UUID) (roachpb.TransactionFingerprintID, bool) { +func (c *fifoCache) get(txnID uuid.UUID) (appstatspb.TransactionFingerprintID, bool) { c.mu.RLock() defer c.mu.RUnlock() diff --git a/pkg/sql/contention/txnidcache/fifo_cache_test.go b/pkg/sql/contention/txnidcache/fifo_cache_test.go index 655c058800d4..6e9dae10bd90 100644 --- a/pkg/sql/contention/txnidcache/fifo_cache_test.go +++ b/pkg/sql/contention/txnidcache/fifo_cache_test.go @@ -17,7 +17,7 @@ import ( "strings" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/datadriven" @@ -27,7 +27,7 @@ import ( func TestFIFOCacheDataDriven(t *testing.T) { var cache *fifoCache inputBlocks := make(map[string]*block) - expectedMaps := make(map[string]map[uuid.UUID]roachpb.TransactionFingerprintID) + expectedMaps := make(map[string]map[uuid.UUID]appstatspb.TransactionFingerprintID) blockToNameMap := make(map[*block]string) datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { @@ -132,18 +132,18 @@ func cloneBlock(b *block) *block { func generateInputBlock( size int, -) (input *block, expected map[uuid.UUID]roachpb.TransactionFingerprintID) { +) (input *block, expected map[uuid.UUID]appstatspb.TransactionFingerprintID) { if size > blockSize { panic(fmt.Sprintf("input block size cannot be greater than %d", blockSize)) } input = &block{} - expected = make(map[uuid.UUID]roachpb.TransactionFingerprintID) + expected = make(map[uuid.UUID]appstatspb.TransactionFingerprintID) for i := 0; i < size; i++ { input[i].TxnID = uuid.FastMakeV4() input[i].TxnFingerprintID = - roachpb.TransactionFingerprintID(rand.Uint64()) + appstatspb.TransactionFingerprintID(rand.Uint64()) expected[input[i].TxnID] = input[i].TxnFingerprintID } diff --git a/pkg/sql/contention/txnidcache/txn_id_cache.go b/pkg/sql/contention/txnidcache/txn_id_cache.go index eeeea37dcbbf..992ce7c232eb 100644 --- a/pkg/sql/contention/txnidcache/txn_id_cache.go +++ b/pkg/sql/contention/txnidcache/txn_id_cache.go @@ -13,8 +13,8 @@ package txnidcache import ( "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -26,7 +26,7 @@ type Reader interface { // Lookup returns the corresponding transaction fingerprint ID for a given txnID, // if the given txnID has no entry in the Cache, the returned "found" boolean // will be false. - Lookup(txnID uuid.UUID) (result roachpb.TransactionFingerprintID, found bool) + Lookup(txnID uuid.UUID) (result appstatspb.TransactionFingerprintID, found bool) } // Writer is the interface that can be used to write to txnidcache. @@ -115,7 +115,7 @@ type Cache struct { var ( entrySize = int64(uuid.UUID{}.Size()) + - roachpb.TransactionFingerprintID(0).Size() + appstatspb.TransactionFingerprintID(0).Size() ) var ( @@ -157,13 +157,13 @@ func (t *Cache) Start(ctx context.Context, stopper *stop.Stopper) { } // Lookup implements the Reader interface. -func (t *Cache) Lookup(txnID uuid.UUID) (result roachpb.TransactionFingerprintID, found bool) { +func (t *Cache) Lookup(txnID uuid.UUID) (result appstatspb.TransactionFingerprintID, found bool) { t.metrics.CacheReadCounter.Inc(1) txnFingerprintID, found := t.store.get(txnID) if !found { t.metrics.CacheMissCounter.Inc(1) - return roachpb.InvalidTransactionFingerprintID, found + return appstatspb.InvalidTransactionFingerprintID, found } return txnFingerprintID, found diff --git a/pkg/sql/contention/txnidcache/txn_id_cache_test.go b/pkg/sql/contention/txnidcache/txn_id_cache_test.go index c58e8bc7d794..fdd8695aa56a 100644 --- a/pkg/sql/contention/txnidcache/txn_id_cache_test.go +++ b/pkg/sql/contention/txnidcache/txn_id_cache_test.go @@ -18,9 +18,9 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contention/txnidcache" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -46,13 +46,13 @@ func TestTransactionIDCache(t *testing.T) { params, _ := tests.CreateTestServerParams() appName := "txnIDCacheTest" - expectedTxnIDToUUIDMapping := make(map[uuid.UUID]roachpb.TransactionFingerprintID) + expectedTxnIDToUUIDMapping := make(map[uuid.UUID]appstatspb.TransactionFingerprintID) injector := runtimeHookInjector{} injector.setHook(func( sessionData *sessiondata.SessionData, txnID uuid.UUID, - txnFingerprintID roachpb.TransactionFingerprintID, + txnFingerprintID appstatspb.TransactionFingerprintID, ) { if strings.Contains(sessionData.ApplicationName, appName) { expectedTxnIDToUUIDMapping[txnID] = txnFingerprintID @@ -216,9 +216,9 @@ func TestTransactionIDCache(t *testing.T) { injector.setHook(func( sessionData *sessiondata.SessionData, txnID uuid.UUID, - txnFingerprintID roachpb.TransactionFingerprintID) { + txnFingerprintID appstatspb.TransactionFingerprintID) { if strings.Contains(sessionData.ApplicationName, appName) { - if txnFingerprintID != roachpb.InvalidTransactionFingerprintID { + if txnFingerprintID != appstatspb.InvalidTransactionFingerprintID { txnIDCache.DrainWriteBuffer() testutils.SucceedsWithin(t, func() error { @@ -227,7 +227,7 @@ func TestTransactionIDCache(t *testing.T) { return errors.Newf("expected provision txn fingerprint id to be found for "+ "txn(%s), but it was not", txnID) } - if existingTxnFingerprintID != roachpb.InvalidTransactionFingerprintID { + if existingTxnFingerprintID != appstatspb.InvalidTransactionFingerprintID { return errors.Newf("expected txn (%s) to have a provisional"+ "txn fingerprint id, but this txn already has a resolved "+ "txn fingerprint id: %d", txnID, existingTxnFingerprintID) @@ -268,11 +268,11 @@ func TestInvalidTxnID(t *testing.T) { inputData := []contentionpb.ResolvedTxnID{ { TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.TransactionFingerprintID(1), + TxnFingerprintID: appstatspb.TransactionFingerprintID(1), }, { TxnID: uuid.FastMakeV4(), - TxnFingerprintID: roachpb.TransactionFingerprintID(2), + TxnFingerprintID: appstatspb.TransactionFingerprintID(2), }, } @@ -316,14 +316,14 @@ type runtimeHookInjector struct { op func( sessionData *sessiondata.SessionData, txnID uuid.UUID, - txnFingerprintID roachpb.TransactionFingerprintID, + txnFingerprintID appstatspb.TransactionFingerprintID, ) } func (s *runtimeHookInjector) hook( sessionData *sessiondata.SessionData, txnID uuid.UUID, - txnFingerprintID roachpb.TransactionFingerprintID, + txnFingerprintID appstatspb.TransactionFingerprintID, ) { s.RLock() defer s.RUnlock() @@ -334,7 +334,7 @@ func (s *runtimeHookInjector) setHook( op func( sessionData *sessiondata.SessionData, txnID uuid.UUID, - txnFingerprintID roachpb.TransactionFingerprintID, + txnFingerprintID appstatspb.TransactionFingerprintID, ), ) { s.Lock() diff --git a/pkg/sql/contention/txnidcache/writer_test.go b/pkg/sql/contention/txnidcache/writer_test.go index 69048c861ca6..cf98aa628f2a 100644 --- a/pkg/sql/contention/txnidcache/writer_test.go +++ b/pkg/sql/contention/txnidcache/writer_test.go @@ -18,8 +18,8 @@ import ( "sync" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -97,7 +97,7 @@ func BenchmarkWriter(b *testing.B) { randomValue := randomValueBase + i w.Record(contentionpb.ResolvedTxnID{ TxnID: generateUUID(uint64(randomValue)), - TxnFingerprintID: roachpb.TransactionFingerprintID(math.MaxInt - randomValue), + TxnFingerprintID: appstatspb.TransactionFingerprintID(math.MaxInt - randomValue), }) } }(writerIdx) diff --git a/pkg/sql/contentionpb/BUILD.bazel b/pkg/sql/contentionpb/BUILD.bazel index 52389c66ed97..9e187080f142 100644 --- a/pkg/sql/contentionpb/BUILD.bazel +++ b/pkg/sql/contentionpb/BUILD.bazel @@ -38,6 +38,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb", # keep + "//pkg/sql/appstatspb", # keep "//pkg/sql/catalog/descpb", # keep "//pkg/sql/clusterunique", # keep "//pkg/util/uuid", # keep diff --git a/pkg/sql/contentionpb/contention.proto b/pkg/sql/contentionpb/contention.proto index 36c860d24841..098d4d0b714e 100644 --- a/pkg/sql/contentionpb/contention.proto +++ b/pkg/sql/contentionpb/contention.proto @@ -140,9 +140,9 @@ message ResolvedTxnID { // txnFingerprintID is the corresponding transaction fingerprint ID of the // txnID. However, if the transaction fingerprint ID of the txnID is not yet // known, (i.e. when the transaction is still executing), this value is filled - // with roachpb.InvalidTransactionFingerprintID. + // with appstatspb.InvalidTransactionFingerprintID. uint64 txnFingerprintID = 2 [(gogoproto.customname) = "TxnFingerprintID", - (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TransactionFingerprintID", + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.TransactionFingerprintID", (gogoproto.nullable) = false]; } @@ -153,7 +153,7 @@ message ExtendedContentionEvent { ]; uint64 blocking_txn_fingerprint_id = 2 [ (gogoproto.customname) = "BlockingTxnFingerprintID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TransactionFingerprintID" + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.TransactionFingerprintID" ]; bytes waiting_txn_id = 3 [ @@ -163,7 +163,7 @@ message ExtendedContentionEvent { ]; uint64 waiting_txn_fingerprint_id = 4 [ (gogoproto.customname) = "WaitingTxnFingerprintID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TransactionFingerprintID" + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.TransactionFingerprintID" ]; google.protobuf.Timestamp collection_ts = 5 [ @@ -173,7 +173,7 @@ message ExtendedContentionEvent { uint64 waiting_stmt_fingerprint_id = 6 [ (gogoproto.customname) = "WaitingStmtFingerprintID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtFingerprintID" + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.StmtFingerprintID" ]; bytes waiting_stmt_id = 7 [(gogoproto.customname) = "WaitingStmtID", diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 9598de358574..92b38c1cf2b8 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" @@ -1291,7 +1292,7 @@ func makeJobsTableRows( // execStatAvg is a helper for execution stats shown in virtual tables. Returns // NULL when the count is 0, or the mean of the given NumericStat. -func execStatAvg(count int64, n roachpb.NumericStat) tree.Datum { +func execStatAvg(count int64, n appstatspb.NumericStat) tree.Datum { if count == 0 { return tree.DNull } @@ -1300,7 +1301,7 @@ func execStatAvg(count int64, n roachpb.NumericStat) tree.Datum { // execStatVar is a helper for execution stats shown in virtual tables. Returns // NULL when the count is 0, or the variance of the given NumericStat. -func execStatVar(count int64, n roachpb.NumericStat) tree.Datum { +func execStatVar(count int64, n appstatspb.NumericStat) tree.Datum { if count == 0 { return tree.DNull } @@ -1392,7 +1393,7 @@ CREATE TABLE crdb_internal.node_statement_statistics ( nodeID, _ := p.execCfg.NodeInfo.NodeID.OptionalNodeID() // zero if not available - statementVisitor := func(_ context.Context, stats *roachpb.CollectedStatementStatistics) error { + statementVisitor := func(_ context.Context, stats *appstatspb.CollectedStatementStatistics) error { anonymized := tree.DNull anonStr, ok := scrubStmtStatKey(p.getVirtualTabler(), stats.Key.Query) if ok { @@ -1421,7 +1422,7 @@ CREATE TABLE crdb_internal.node_statement_statistics ( } txnFingerprintID := tree.DNull - if stats.Key.TransactionFingerprintID != roachpb.InvalidTransactionFingerprintID { + if stats.Key.TransactionFingerprintID != appstatspb.InvalidTransactionFingerprintID { txnFingerprintID = tree.NewDString(strconv.FormatUint(uint64(stats.Key.TransactionFingerprintID), 10)) } @@ -1554,7 +1555,7 @@ CREATE TABLE crdb_internal.node_transaction_statistics ( nodeID, _ := p.execCfg.NodeInfo.NodeID.OptionalNodeID() // zero if not available - transactionVisitor := func(_ context.Context, stats *roachpb.CollectedTransactionStatistics) error { + transactionVisitor := func(_ context.Context, stats *appstatspb.CollectedTransactionStatistics) error { stmtFingerprintIDsDatum := tree.NewDArray(types.String) for _, stmtFingerprintID := range stats.StatementFingerprintIDs { if err := stmtFingerprintIDsDatum.Append(tree.NewDString(strconv.FormatUint(uint64(stmtFingerprintID), 10))); err != nil { @@ -1632,7 +1633,7 @@ CREATE TABLE crdb_internal.node_txn_stats ( nodeID, _ := p.execCfg.NodeInfo.NodeID.OptionalNodeID() // zero if not available - appTxnStatsVisitor := func(appName string, stats *roachpb.TxnStats) error { + appTxnStatsVisitor := func(appName string, stats *appstatspb.TxnStats) error { return addRow( tree.NewDInt(tree.DInt(nodeID)), tree.NewDString(appName), @@ -6015,7 +6016,7 @@ CREATE TABLE crdb_internal.cluster_statement_statistics ( return memSQLStats.IterateStatementStats(ctx, &sqlstats.IteratorOptions{ SortedAppNames: true, SortedKey: true, - }, func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + }, func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { aggregatedTs, err := tree.MakeDTimestampTZ(curAggTs, time.Microsecond) if err != nil { @@ -6250,7 +6251,7 @@ CREATE TABLE crdb_internal.cluster_transaction_statistics ( SortedKey: true, }, func( ctx context.Context, - statistics *roachpb.CollectedTransactionStatistics) error { + statistics *appstatspb.CollectedTransactionStatistics) error { aggregatedTs, err := tree.MakeDTimestampTZ(curAggTs, time.Microsecond) if err != nil { diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index f26b52e52b96..a3ceab27db4c 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -16,8 +16,8 @@ import ( "fmt" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execstats" @@ -162,7 +162,7 @@ func (p *planner) maybeLogStatement( queryReceived time.Time, hasAdminRoleCache *HasAdminRoleCache, telemetryLoggingMetrics *TelemetryLoggingMetrics, - stmtFingerprintID roachpb.StmtFingerprintID, + stmtFingerprintID appstatspb.StmtFingerprintID, queryStats *topLevelQueryStats, ) { p.maybeLogStatementInternal(ctx, execType, isCopy, numRetries, txnCounter, rows, err, queryReceived, hasAdminRoleCache, telemetryLoggingMetrics, stmtFingerprintID, queryStats) @@ -179,7 +179,7 @@ func (p *planner) maybeLogStatementInternal( startTime time.Time, hasAdminRoleCache *HasAdminRoleCache, telemetryMetrics *TelemetryLoggingMetrics, - stmtFingerprintID roachpb.StmtFingerprintID, + stmtFingerprintID appstatspb.StmtFingerprintID, queryStats *topLevelQueryStats, ) { // Note: if you find the code below crashing because p.execCfg == nil, diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 948c264180f6..d2525cec85f0 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -55,6 +55,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" @@ -1533,7 +1534,7 @@ type ExecutorTestingKnobs struct { BeforeTxnStatsRecorded func( sessionData *sessiondata.SessionData, txnID uuid.UUID, - txnFingerprintID roachpb.TransactionFingerprintID, + txnFingerprintID appstatspb.TransactionFingerprintID, ) // AfterBackupCheckpoint if set will be called after a BACKUP-CHECKPOINT @@ -3406,9 +3407,9 @@ func (m *sessionDataMutator) SetOptimizerUseLimitOrderingForStreamingGroupBy(val // Utility functions related to scrubbing sensitive information on SQL Stats. // quantizeCounts ensures that the Count field in the -// roachpb.StatementStatistics is bucketed to the order of magnitude base 10s +// appstatspb.StatementStatistics is bucketed to the order of magnitude base 10s // and recomputes the squared differences using the new Count value. -func quantizeCounts(d *roachpb.StatementStatistics) { +func quantizeCounts(d *appstatspb.StatementStatistics) { oldCount := d.Count newCount := telemetry.Bucket10(oldCount) d.Count = newCount diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index e9cfcdf12c95..b7f2b514b919 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -14,7 +14,7 @@ import ( "context" "strconv" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/idxrecommendations" @@ -121,7 +121,7 @@ func (ex *connExecutor) recordStatementSummary( rowsAffected int, stmtErr error, stats topLevelQueryStats, -) roachpb.StmtFingerprintID { +) appstatspb.StmtFingerprintID { phaseTimes := ex.statsCollector.PhaseTimes() // Collect the statistics. @@ -163,7 +163,7 @@ func (ex *connExecutor) recordStatementSummary( } fullScan := flags.IsSet(planFlagContainsFullIndexScan) || flags.IsSet(planFlagContainsFullTableScan) - recordedStmtStatsKey := roachpb.StatementStatisticsKey{ + recordedStmtStatsKey := appstatspb.StatementStatisticsKey{ Query: stmt.StmtNoConstants, QuerySummary: stmt.StmtSummary, DistSQL: flags.IsDistributed(), diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index 81f95b2186ce..b982a35fbb70 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -19,9 +19,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/multitenant" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -515,7 +515,7 @@ func (ih *instrumentationHelper) RecordPlanInfo( // PlanForStats returns the plan as an ExplainTreePlanNode tree, if it was // collected (nil otherwise). It should be called after RecordExplainPlan() and // RecordPlanInfo(). -func (ih *instrumentationHelper) PlanForStats(ctx context.Context) *roachpb.ExplainTreePlanNode { +func (ih *instrumentationHelper) PlanForStats(ctx context.Context) *appstatspb.ExplainTreePlanNode { if ih.explainPlan == nil || !ih.savePlanForStats { return nil } diff --git a/pkg/sql/instrumentation_test.go b/pkg/sql/instrumentation_test.go index 0cdc9508c8e6..a2aba5c80ad5 100644 --- a/pkg/sql/instrumentation_test.go +++ b/pkg/sql/instrumentation_test.go @@ -16,8 +16,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -47,21 +47,21 @@ func TestSampledStatsCollection(t *testing.T) { stmt string, implicitTxn bool, database string, - ) *roachpb.CollectedStatementStatistics { + ) *appstatspb.CollectedStatementStatistics { t.Helper() - key := roachpb.StatementStatisticsKey{ + key := appstatspb.StatementStatisticsKey{ Query: stmt, ImplicitTxn: implicitTxn, Database: database, Failed: false, } - var stats *roachpb.CollectedStatementStatistics + var stats *appstatspb.CollectedStatementStatistics require.NoError(t, server.SQLServer().(*Server).sqlStats. GetLocalMemProvider(). IterateStatementStats( ctx, &sqlstats.IteratorOptions{}, - func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { if statistics.Key.Query == key.Query && statistics.Key.ImplicitTxn == key.ImplicitTxn && statistics.Key.Database == key.Database && @@ -80,17 +80,17 @@ func TestSampledStatsCollection(t *testing.T) { getTxnStats := func( t *testing.T, server serverutils.TestServerInterface, - key roachpb.TransactionFingerprintID, - ) *roachpb.CollectedTransactionStatistics { + key appstatspb.TransactionFingerprintID, + ) *appstatspb.CollectedTransactionStatistics { t.Helper() - var stats *roachpb.CollectedTransactionStatistics + var stats *appstatspb.CollectedTransactionStatistics require.NoError(t, server.SQLServer().(*Server).sqlStats. GetLocalMemProvider(). IterateTransactionStats( ctx, &sqlstats.IteratorOptions{}, - func(ctx context.Context, statistics *roachpb.CollectedTransactionStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedTransactionStatistics) error { if statistics.TransactionFingerprintID == key { stats = statistics } @@ -169,7 +169,7 @@ func TestSampledStatsCollection(t *testing.T) { key := util.MakeFNV64() key.Add(uint64(aggStats.ID)) key.Add(uint64(selectStats.ID)) - txStats := getTxnStats(t, s, roachpb.TransactionFingerprintID(key.Sum())) + txStats := getTxnStats(t, s, appstatspb.TransactionFingerprintID(key.Sum())) require.Equal(t, int64(2), txStats.Stats.Count, "expected to have collected two sets of general stats") require.Equal(t, int64(1), txStats.Stats.ExecStats.Count, "expected to have collected exactly one set of execution stats") diff --git a/pkg/sql/opt/exec/explain/BUILD.bazel b/pkg/sql/opt/exec/explain/BUILD.bazel index c642c53ae436..cc7c28a5ae2d 100644 --- a/pkg/sql/opt/exec/explain/BUILD.bazel +++ b/pkg/sql/opt/exec/explain/BUILD.bazel @@ -19,6 +19,7 @@ go_library( deps = [ "//pkg/geo/geoindex", "//pkg/roachpb", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/inverted", # keep diff --git a/pkg/sql/opt/exec/explain/output.go b/pkg/sql/opt/exec/explain/output.go index fbbfc7ada45e..647ce7a4d4d2 100644 --- a/pkg/sql/opt/exec/explain/output.go +++ b/pkg/sql/opt/exec/explain/output.go @@ -16,7 +16,7 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -234,23 +234,23 @@ func (ob *OutputBuilder) BuildString() string { } // BuildProtoTree creates a representation of the plan as a tree of -// roachpb.ExplainTreePlanNodes. -func (ob *OutputBuilder) BuildProtoTree() *roachpb.ExplainTreePlanNode { +// appstatspb.ExplainTreePlanNodes. +func (ob *OutputBuilder) BuildProtoTree() *appstatspb.ExplainTreePlanNode { // We reconstruct the hierarchy using the levels. // stack keeps track of the current node on each level. We use a sentinel node // for level 0. - sentinel := &roachpb.ExplainTreePlanNode{} - stack := []*roachpb.ExplainTreePlanNode{sentinel} + sentinel := &appstatspb.ExplainTreePlanNode{} + stack := []*appstatspb.ExplainTreePlanNode{sentinel} for _, entry := range ob.entries { if entry.isNode() { parent := stack[entry.level-1] - child := &roachpb.ExplainTreePlanNode{Name: entry.node} + child := &appstatspb.ExplainTreePlanNode{Name: entry.node} parent.Children = append(parent.Children, child) stack = append(stack[:entry.level], child) } else { node := stack[len(stack)-1] - node.Attrs = append(node.Attrs, &roachpb.ExplainTreePlanNode_Attr{ + node.Attrs = append(node.Attrs, &appstatspb.ExplainTreePlanNode_Attr{ Key: entry.field, Value: entry.fieldVal, }) diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 6eb2ff4cb876..0a383064e8e1 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -56,6 +56,7 @@ go_library( "//pkg/server/telemetry", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/descpb", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 3167f884b9ca..ac70d2bb3dae 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/randgen/randgencfg" @@ -4213,12 +4214,12 @@ value if you rely on the HLC for accuracy.`, ReturnType: tree.FixedReturnType(types.Jsonb), Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { arr := tree.MustBeDArray(args[0]) - var aggregatedStats roachpb.StatementStatistics + var aggregatedStats appstatspb.StatementStatistics for _, statsDatum := range arr.Array { if statsDatum == tree.DNull { continue } - var stats roachpb.StatementStatistics + var stats appstatspb.StatementStatistics statsJSON := tree.MustBeDJSON(statsDatum).JSON if err := sqlstatsutil.DecodeStmtStatsStatisticsJSON(statsJSON, &stats); err != nil { return nil, err @@ -4234,7 +4235,7 @@ value if you rely on the HLC for accuracy.`, return tree.NewDJSON(aggregatedJSON), nil }, - Info: "Merge an array of roachpb.StatementStatistics into a single JSONB object", + Info: "Merge an array of appstatspb.StatementStatistics into a single JSONB object", Volatility: volatility.Immutable, }, ), @@ -4244,12 +4245,12 @@ value if you rely on the HLC for accuracy.`, ReturnType: tree.FixedReturnType(types.Jsonb), Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { arr := tree.MustBeDArray(args[0]) - var aggregatedStats roachpb.TransactionStatistics + var aggregatedStats appstatspb.TransactionStatistics for _, statsDatum := range arr.Array { if statsDatum == tree.DNull { continue } - var stats roachpb.TransactionStatistics + var stats appstatspb.TransactionStatistics statsJSON := tree.MustBeDJSON(statsDatum).JSON if err := sqlstatsutil.DecodeTxnStatsStatisticsJSON(statsJSON, &stats); err != nil { return nil, err @@ -4259,7 +4260,7 @@ value if you rely on the HLC for accuracy.`, } aggregatedJSON, err := sqlstatsutil.BuildTxnStatisticsJSON( - &roachpb.CollectedTransactionStatistics{ + &appstatspb.CollectedTransactionStatistics{ Stats: aggregatedStats, }) if err != nil { @@ -4268,7 +4269,7 @@ value if you rely on the HLC for accuracy.`, return tree.NewDJSON(aggregatedJSON), nil }, - Info: "Merge an array of roachpb.TransactionStatistics into a single JSONB object", + Info: "Merge an array of appstatspb.TransactionStatistics into a single JSONB object", Volatility: volatility.Immutable, }, ), @@ -4278,14 +4279,14 @@ value if you rely on the HLC for accuracy.`, ReturnType: tree.FixedReturnType(types.Jsonb), Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { arr := tree.MustBeDArray(args[0]) - metadata := &roachpb.AggregatedStatementMetadata{} + metadata := &appstatspb.AggregatedStatementMetadata{} for _, metadataDatum := range arr.Array { if metadataDatum == tree.DNull { continue } - var statistics roachpb.CollectedStatementStatistics + var statistics appstatspb.CollectedStatementStatistics metadataJSON := tree.MustBeDJSON(metadataDatum).JSON err := sqlstatsutil.DecodeStmtStatsMetadataJSON(metadataJSON, &statistics) if err != nil { diff --git a/pkg/sql/sqlstats/BUILD.bazel b/pkg/sql/sqlstats/BUILD.bazel index 82e704104c98..d4222fe29963 100644 --- a/pkg/sql/sqlstats/BUILD.bazel +++ b/pkg/sql/sqlstats/BUILD.bazel @@ -13,6 +13,7 @@ go_library( deps = [ "//pkg/roachpb", "//pkg/settings", + "//pkg/sql/appstatspb", "//pkg/sql/clusterunique", "//pkg/sql/execstats", "//pkg/sql/sem/tree", diff --git a/pkg/sql/sqlstats/insights/BUILD.bazel b/pkg/sql/sqlstats/insights/BUILD.bazel index 89033421da7e..a659cbe06640 100644 --- a/pkg/sql/sqlstats/insights/BUILD.bazel +++ b/pkg/sql/sqlstats/insights/BUILD.bazel @@ -20,9 +20,9 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/clusterunique", "//pkg/sql/contention/contentionutils", "//pkg/util/cache", @@ -50,8 +50,8 @@ go_test( args = ["-test.timeout=295s"], embed = [":insights"], deps = [ - "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/clusterunique", "//pkg/util/stop", "//pkg/util/uint128", diff --git a/pkg/sql/sqlstats/insights/detector.go b/pkg/sql/sqlstats/insights/detector.go index 518ce44c151c..1e11904b31fe 100644 --- a/pkg/sql/sqlstats/insights/detector.go +++ b/pkg/sql/sqlstats/insights/detector.go @@ -13,8 +13,8 @@ package insights import ( "container/list" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/util/quantile" ) @@ -56,11 +56,11 @@ type anomalyDetector struct { settings *cluster.Settings metrics Metrics store *list.List - index map[roachpb.StmtFingerprintID]*list.Element + index map[appstatspb.StmtFingerprintID]*list.Element } type latencySummaryEntry struct { - key roachpb.StmtFingerprintID + key appstatspb.StmtFingerprintID value *quantile.Stream } @@ -126,7 +126,7 @@ func newAnomalyDetector(settings *cluster.Settings, metrics Metrics) *anomalyDet settings: settings, metrics: metrics, store: list.New(), - index: make(map[roachpb.StmtFingerprintID]*list.Element), + index: make(map[appstatspb.StmtFingerprintID]*list.Element), } } diff --git a/pkg/sql/sqlstats/insights/detector_test.go b/pkg/sql/sqlstats/insights/detector_test.go index 4fc25f8e851d..0c2d26fae5d0 100644 --- a/pkg/sql/sqlstats/insights/detector_test.go +++ b/pkg/sql/sqlstats/insights/detector_test.go @@ -16,8 +16,8 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/stretchr/testify/require" ) @@ -173,7 +173,7 @@ func TestLatencyQuantileDetector(t *testing.T) { for i := 0; i < test.fingerprints; i++ { d.isSlow(&Statement{ LatencyInSeconds: AnomalyDetectionLatencyThreshold.Get(&st.SV).Seconds(), - FingerprintID: roachpb.StmtFingerprintID(i), + FingerprintID: appstatspb.StmtFingerprintID(i), }) } test.assertion(t, metrics) diff --git a/pkg/sql/sqlstats/insights/insights.proto b/pkg/sql/sqlstats/insights/insights.proto index aafeaa8bd1ff..effb8556a8e3 100644 --- a/pkg/sql/sqlstats/insights/insights.proto +++ b/pkg/sql/sqlstats/insights/insights.proto @@ -66,7 +66,7 @@ message Transaction { (gogoproto.nullable) = false]; uint64 fingerprint_id = 2 [(gogoproto.customname) = "FingerprintID", - (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TransactionFingerprintID", + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.TransactionFingerprintID", (gogoproto.nullable) = false]; string user_priority = 3; bool implicit_txn = 4; @@ -96,7 +96,7 @@ message Statement { (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/clusterunique.ID", (gogoproto.nullable) = false]; uint64 fingerprint_id = 2 [(gogoproto.customname) = "FingerprintID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtFingerprintID"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/appstatspb.StmtFingerprintID"]; double latency_in_seconds = 3; string query = 4; Status status = 5; diff --git a/pkg/sql/sqlstats/insights/insights_test.go b/pkg/sql/sqlstats/insights/insights_test.go index d40b7cde9e38..737d3f725505 100644 --- a/pkg/sql/sqlstats/insights/insights_test.go +++ b/pkg/sql/sqlstats/insights/insights_test.go @@ -18,8 +18,8 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -62,7 +62,7 @@ func BenchmarkInsights(b *testing.B) { for j := 0; j < numTransactionsPerSession; j++ { statements[numTransactionsPerSession*i+j] = insights.Statement{ // Spread across 6 different statement fingerprints. - FingerprintID: roachpb.StmtFingerprintID(j % 6), + FingerprintID: appstatspb.StmtFingerprintID(j % 6), // Choose latencies in 20ms, 40ms, 60ms, 80ms, 100ms, 120ms, 140ms. // As configured above, only latencies >=100ms are noteworthy. // Since 7 is relatively prime to 6, we'll spread these across all fingerprints. diff --git a/pkg/sql/sqlstats/insights/registry_test.go b/pkg/sql/sqlstats/insights/registry_test.go index 573888699024..c1ae8dca95e8 100644 --- a/pkg/sql/sqlstats/insights/registry_test.go +++ b/pkg/sql/sqlstats/insights/registry_test.go @@ -17,8 +17,8 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/stretchr/testify/require" @@ -42,7 +42,7 @@ func TestRegistry(t *testing.T) { statement := &Statement{ Status: Statement_Completed, ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")), - FingerprintID: roachpb.StmtFingerprintID(100), + FingerprintID: appstatspb.StmtFingerprintID(100), LatencyInSeconds: 2, } expectedStatement := @@ -77,7 +77,7 @@ func TestRegistry(t *testing.T) { // We'll be coming back to build a better failure story for 23.1. statement := &Statement{ ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")), - FingerprintID: roachpb.StmtFingerprintID(100), + FingerprintID: appstatspb.StmtFingerprintID(100), LatencyInSeconds: 2, Status: Statement_Failed, } @@ -112,7 +112,7 @@ func TestRegistry(t *testing.T) { statement := &Statement{ Status: Statement_Completed, ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")), - FingerprintID: roachpb.StmtFingerprintID(100), + FingerprintID: appstatspb.StmtFingerprintID(100), LatencyInSeconds: 2, } st := cluster.MakeTestingClusterSettings() @@ -137,7 +137,7 @@ func TestRegistry(t *testing.T) { LatencyThreshold.Override(ctx, &st.SV, 1*time.Second) statement2 := &Statement{ ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")), - FingerprintID: roachpb.StmtFingerprintID(100), + FingerprintID: appstatspb.StmtFingerprintID(100), LatencyInSeconds: 0.5, } store := newStore(st) @@ -159,14 +159,14 @@ func TestRegistry(t *testing.T) { statement := &Statement{ Status: Statement_Completed, ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")), - FingerprintID: roachpb.StmtFingerprintID(100), + FingerprintID: appstatspb.StmtFingerprintID(100), LatencyInSeconds: 2, } otherSession := Session{ID: clusterunique.IDFromBytes([]byte("cccccccccccccccccccccccccccccccc"))} otherTransaction := &Transaction{ID: uuid.FastMakeV4()} otherStatement := &Statement{ ID: clusterunique.IDFromBytes([]byte("dddddddddddddddddddddddddddddddd")), - FingerprintID: roachpb.StmtFingerprintID(101), + FingerprintID: appstatspb.StmtFingerprintID(101), LatencyInSeconds: 3, } @@ -212,12 +212,12 @@ func TestRegistry(t *testing.T) { statement := &Statement{ Status: Statement_Completed, ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")), - FingerprintID: roachpb.StmtFingerprintID(100), + FingerprintID: appstatspb.StmtFingerprintID(100), LatencyInSeconds: 2, } siblingStatment := &Statement{ ID: clusterunique.IDFromBytes([]byte("dddddddddddddddddddddddddddddddd")), - FingerprintID: roachpb.StmtFingerprintID(101), + FingerprintID: appstatspb.StmtFingerprintID(101), } st := cluster.MakeTestingClusterSettings() @@ -263,7 +263,7 @@ func TestRegistry(t *testing.T) { statement := &Statement{ Status: Statement_Completed, ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")), - FingerprintID: roachpb.StmtFingerprintID(100), + FingerprintID: appstatspb.StmtFingerprintID(100), LatencyInSeconds: 0.00001, } txnHighContention := &Transaction{ID: uuid.FastMakeV4(), Contention: &contentionDuration} diff --git a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel index 78c3f853731a..6bfd05a15d72 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel +++ b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel @@ -27,12 +27,12 @@ go_library( "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", - "//pkg/roachpb", "//pkg/scheduledjobs", "//pkg/security/username", "//pkg/server/serverpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/systemschema", "//pkg/sql/isql", "//pkg/sql/sem/tree", @@ -84,6 +84,7 @@ go_test( "//pkg/security/username", "//pkg/server", "//pkg/sql", + "//pkg/sql/appstatspb", "//pkg/sql/catalog", "//pkg/sql/catalog/systemschema", "//pkg/sql/isql", diff --git a/pkg/sql/sqlstats/persistedsqlstats/appStats.go b/pkg/sql/sqlstats/persistedsqlstats/appStats.go index 108782a26761..70e24c2b74a4 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/appStats.go +++ b/pkg/sql/sqlstats/persistedsqlstats/appStats.go @@ -13,7 +13,7 @@ package persistedsqlstats import ( "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/ssmemstorage" "github.com/cockroachdb/errors" @@ -34,9 +34,9 @@ var _ sqlstats.ApplicationStats = &ApplicationStats{} // RecordStatement implements sqlstats.ApplicationStats interface. func (s *ApplicationStats) RecordStatement( - ctx context.Context, key roachpb.StatementStatisticsKey, value sqlstats.RecordedStmtStats, -) (roachpb.StmtFingerprintID, error) { - var fingerprintID roachpb.StmtFingerprintID + ctx context.Context, key appstatspb.StatementStatisticsKey, value sqlstats.RecordedStmtStats, +) (appstatspb.StmtFingerprintID, error) { + var fingerprintID appstatspb.StmtFingerprintID err := s.recordStatsOrSendMemoryPressureSignal(func() (err error) { fingerprintID, err = s.ApplicationStats.RecordStatement(ctx, key, value) return err @@ -54,7 +54,7 @@ func (s *ApplicationStats) ShouldSample( // RecordTransaction implements sqlstats.ApplicationStats interface and saves // per-transaction statistics. func (s *ApplicationStats) RecordTransaction( - ctx context.Context, key roachpb.TransactionFingerprintID, value sqlstats.RecordedTxnStats, + ctx context.Context, key appstatspb.TransactionFingerprintID, value sqlstats.RecordedTxnStats, ) error { return s.recordStatsOrSendMemoryPressureSignal(func() error { return s.ApplicationStats.RecordTransaction(ctx, key, value) diff --git a/pkg/sql/sqlstats/persistedsqlstats/combined_iterator.go b/pkg/sql/sqlstats/persistedsqlstats/combined_iterator.go index 545f6b2c8544..99f49dbd01e0 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/combined_iterator.go +++ b/pkg/sql/sqlstats/persistedsqlstats/combined_iterator.go @@ -14,7 +14,7 @@ import ( "context" "strings" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/errors" ) @@ -23,7 +23,7 @@ import ( // in-memory and persisted stmt stats provided by the in-memory iterator and // the on-disk iterator. type CombinedStmtStatsIterator struct { - nextToRead *roachpb.CollectedStatementStatistics + nextToRead *appstatspb.CollectedStatementStatistics expectedColCnt int mem struct { @@ -157,13 +157,13 @@ func (c *CombinedStmtStatsIterator) Next(ctx context.Context) (bool, error) { return true, nil } -// Cur returns the roachpb.CollectedStatementStatistics at the current internal +// Cur returns the appstatspb.CollectedStatementStatistics at the current internal // counter. -func (c *CombinedStmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { +func (c *CombinedStmtStatsIterator) Cur() *appstatspb.CollectedStatementStatistics { return c.nextToRead } -func compareStmtStats(lhs, rhs *roachpb.CollectedStatementStatistics) int { +func compareStmtStats(lhs, rhs *appstatspb.CollectedStatementStatistics) int { // 1. we compare their aggregated_ts if lhs.AggregatedTs.Before(rhs.AggregatedTs) { return -1 @@ -209,7 +209,7 @@ func compareStmtStats(lhs, rhs *roachpb.CollectedStatementStatistics) int { // in-memory and persisted txn stats provided by the in-memory iterator and // the on-disk iterator. type CombinedTxnStatsIterator struct { - nextToReadVal *roachpb.CollectedTransactionStatistics + nextToReadVal *appstatspb.CollectedTransactionStatistics expectedColCnt int mem struct { @@ -343,13 +343,13 @@ func (c *CombinedTxnStatsIterator) Next(ctx context.Context) (bool, error) { return true, nil } -// Cur returns the roachpb.CollectedTransactionStatistics at the current internal +// Cur returns the appstatspb.CollectedTransactionStatistics at the current internal // counter. -func (c *CombinedTxnStatsIterator) Cur() *roachpb.CollectedTransactionStatistics { +func (c *CombinedTxnStatsIterator) Cur() *appstatspb.CollectedTransactionStatistics { return c.nextToReadVal } -func compareTxnStats(lhs, rhs *roachpb.CollectedTransactionStatistics) int { +func compareTxnStats(lhs, rhs *appstatspb.CollectedTransactionStatistics) int { // 1. we compare their aggregated_ts if lhs.AggregatedTs.Before(rhs.AggregatedTs) { return -1 diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush.go b/pkg/sql/sqlstats/persistedsqlstats/flush.go index 13963c8bfdc0..1ae5fff82e3e 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush.go @@ -16,7 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -80,7 +80,7 @@ func (s *PersistedSQLStats) flushStmtStats(ctx context.Context, aggregatedTs tim // s.doFlush directly logs errors if they are encountered. Therefore, // no error is returned here. _ = s.SQLStats.IterateStatementStats(ctx, &sqlstats.IteratorOptions{}, - func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { s.doFlush(ctx, func() error { return s.doFlushSingleStmtStats(ctx, statistics, aggregatedTs) }, "failed to flush statement statistics" /* errMsg */) @@ -95,7 +95,7 @@ func (s *PersistedSQLStats) flushStmtStats(ctx context.Context, aggregatedTs tim func (s *PersistedSQLStats) flushTxnStats(ctx context.Context, aggregatedTs time.Time) { _ = s.SQLStats.IterateTransactionStats(ctx, &sqlstats.IteratorOptions{}, - func(ctx context.Context, statistics *roachpb.CollectedTransactionStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedTransactionStatistics) error { s.doFlush(ctx, func() error { return s.doFlushSingleTxnStats(ctx, statistics, aggregatedTs) }, "failed to flush transaction statistics" /* errMsg */) @@ -126,7 +126,7 @@ func (s *PersistedSQLStats) doFlush(ctx context.Context, workFn func() error, er } func (s *PersistedSQLStats) doFlushSingleTxnStats( - ctx context.Context, stats *roachpb.CollectedTransactionStatistics, aggregatedTs time.Time, + ctx context.Context, stats *appstatspb.CollectedTransactionStatistics, aggregatedTs time.Time, ) error { return s.cfg.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { // Explicitly copy the stats variable so the txn closure is retryable. @@ -149,7 +149,7 @@ func (s *PersistedSQLStats) doFlushSingleTxnStats( } readFn := func(ctx context.Context, txn isql.Txn) error { - persistedData := roachpb.TransactionStatistics{} + persistedData := appstatspb.TransactionStatistics{} err := s.fetchPersistedTransactionStats(ctx, txn, aggregatedTs, serializedFingerprintID, scopedStats.App, &persistedData) if err != nil { return err @@ -172,7 +172,7 @@ func (s *PersistedSQLStats) doFlushSingleTxnStats( } func (s *PersistedSQLStats) doFlushSingleStmtStats( - ctx context.Context, stats *roachpb.CollectedStatementStatistics, aggregatedTs time.Time, + ctx context.Context, stats *appstatspb.CollectedStatementStatistics, aggregatedTs time.Time, ) error { return s.cfg.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { // Explicitly copy the stats so that this closure is retryable. @@ -205,7 +205,7 @@ func (s *PersistedSQLStats) doFlushSingleStmtStats( } readFn := func(ctx context.Context, txn isql.Txn) error { - persistedData := roachpb.StatementStatistics{} + persistedData := appstatspb.StatementStatistics{} err := s.fetchPersistedStatementStats( ctx, txn, @@ -301,7 +301,7 @@ func (s *PersistedSQLStats) insertTransactionStats( txn isql.Txn, aggregatedTs time.Time, serializedFingerprintID []byte, - stats *roachpb.CollectedTransactionStatistics, + stats *appstatspb.CollectedTransactionStatistics, ) (rowsAffected int, err error) { insertStmt := ` INSERT INTO system.transaction_statistics @@ -348,7 +348,7 @@ func (s *PersistedSQLStats) updateTransactionStats( txn isql.Txn, aggregatedTs time.Time, serializedFingerprintID []byte, - stats *roachpb.CollectedTransactionStatistics, + stats *appstatspb.CollectedTransactionStatistics, ) error { updateStmt := ` UPDATE system.transaction_statistics @@ -398,7 +398,7 @@ func (s *PersistedSQLStats) updateStatementStats( serializedFingerprintID []byte, serializedTransactionFingerprintID []byte, serializedPlanHash []byte, - stats *roachpb.CollectedStatementStatistics, + stats *appstatspb.CollectedStatementStatistics, ) error { updateStmt := ` UPDATE system.statement_statistics @@ -466,7 +466,7 @@ func (s *PersistedSQLStats) insertStatementStats( serializedFingerprintID []byte, serializedTransactionFingerprintID []byte, serializedPlanHash []byte, - stats *roachpb.CollectedStatementStatistics, + stats *appstatspb.CollectedStatementStatistics, ) (rowsAffected int, err error) { aggInterval := s.GetAggregationInterval() @@ -536,7 +536,7 @@ func (s *PersistedSQLStats) fetchPersistedTransactionStats( aggregatedTs time.Time, serializedFingerprintID []byte, appName string, - result *roachpb.TransactionStatistics, + result *appstatspb.TransactionStatistics, ) error { // We use `SELECT ... FOR UPDATE` statement because we are going to perform // and `UPDATE` on the stats for the given fingerprint later. @@ -592,8 +592,8 @@ func (s *PersistedSQLStats) fetchPersistedStatementStats( serializedFingerprintID []byte, serializedTransactionFingerprintID []byte, serializedPlanHash []byte, - key *roachpb.StatementStatisticsKey, - result *roachpb.StatementStatistics, + key *appstatspb.StatementStatisticsKey, + result *appstatspb.StatementStatistics, ) error { readStmt := ` SELECT diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush_test.go b/pkg/sql/sqlstats/persistedsqlstats/flush_test.go index 8c5db541cf50..37711ea8b67b 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -580,7 +581,7 @@ func verifyInMemoryStatsCorrectness( t *testing.T, tcs []testCase, statsProvider *persistedsqlstats.PersistedSQLStats, ) { for _, tc := range tcs { - err := statsProvider.SQLStats.IterateStatementStats(context.Background(), &sqlstats.IteratorOptions{}, func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + err := statsProvider.SQLStats.IterateStatementStats(context.Background(), &sqlstats.IteratorOptions{}, func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { if tc.fingerprint == statistics.Key.Query { require.Equal(t, tc.count, statistics.Stats.Count, "fingerprint: %s", tc.fingerprint) } @@ -595,7 +596,7 @@ func verifyInMemoryStatsEmpty( t *testing.T, tcs []testCase, statsProvider *persistedsqlstats.PersistedSQLStats, ) { for _, tc := range tcs { - err := statsProvider.SQLStats.IterateStatementStats(context.Background(), &sqlstats.IteratorOptions{}, func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + err := statsProvider.SQLStats.IterateStatementStats(context.Background(), &sqlstats.IteratorOptions{}, func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { if tc.fingerprint == statistics.Key.Query { require.Equal(t, 0 /* expected */, statistics.Stats.Count, "fingerprint: %s", tc.fingerprint) } diff --git a/pkg/sql/sqlstats/persistedsqlstats/mem_iterator.go b/pkg/sql/sqlstats/persistedsqlstats/mem_iterator.go index 57027b80ea3c..4e74c7ec907b 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/mem_iterator.go +++ b/pkg/sql/sqlstats/persistedsqlstats/mem_iterator.go @@ -13,7 +13,7 @@ package persistedsqlstats import ( "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal" ) @@ -23,7 +23,7 @@ import ( // fields populated, memStmtStatsIterator overrides the // sslocal.StmtStatsIterator's Cur() method to populate the aggregated_ts // and aggregation_interval fields on the returning -// roachpb.CollectedStatementStatistics. +// appstatspb.CollectedStatementStatistics. type memStmtStatsIterator struct { *sslocal.StmtStatsIterator aggregatedTs time.Time @@ -45,7 +45,7 @@ func newMemStmtStatsIterator( // Cur calls the m.StmtStatsIterator.Cur() and populates the c.AggregatedTs // field and c.AggregationInterval field. -func (m *memStmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { +func (m *memStmtStatsIterator) Cur() *appstatspb.CollectedStatementStatistics { c := m.StmtStatsIterator.Cur() c.AggregatedTs = m.aggregatedTs c.AggregationInterval = m.aggInterval @@ -57,7 +57,7 @@ func (m *memStmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { // fields populated, memTxnStatsIterator overrides the // sslocal.TxnStatsIterator's Cur() method to populate the aggregated_ts and // aggregatoin_interval fields fields on the returning -// roachpb.CollectedTransactionStatistics. +// appstatspb.CollectedTransactionStatistics. type memTxnStatsIterator struct { *sslocal.TxnStatsIterator aggregatedTs time.Time @@ -79,7 +79,7 @@ func newMemTxnStatsIterator( // Cur calls the m.TxnStatsIterator.Cur() and populates the stats.AggregatedTs // and stats.AggregationInterval fields. -func (m *memTxnStatsIterator) Cur() *roachpb.CollectedTransactionStatistics { +func (m *memTxnStatsIterator) Cur() *appstatspb.CollectedTransactionStatistics { stats := m.TxnStatsIterator.Cur() stats.AggregatedTs = m.aggregatedTs stats.AggregationInterval = m.aggInterval diff --git a/pkg/sql/sqlstats/persistedsqlstats/reader_test.go b/pkg/sql/sqlstats/persistedsqlstats/reader_test.go index 75bd15527702..5e90dbd48a5a 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/reader_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/reader_test.go @@ -19,8 +19,8 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -88,7 +88,7 @@ func TestPersistedSQLStatsRead(t *testing.T) { foundQueries := make(map[string]struct{}) foundTxns := make(map[string]struct{}) - stmtFingerprintIDToQueries := make(map[roachpb.StmtFingerprintID]string) + stmtFingerprintIDToQueries := make(map[appstatspb.StmtFingerprintID]string) require.NoError(t, sqlStats.IterateStatementStats( @@ -97,7 +97,7 @@ func TestPersistedSQLStatsRead(t *testing.T) { SortedKey: true, SortedAppNames: true, }, - func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { if expectedExecCount, ok := expectedStmtFingerprints[statistics.Key.Query]; ok { _, ok = foundQueries[statistics.Key.Query] require.False( @@ -118,7 +118,7 @@ func TestPersistedSQLStatsRead(t *testing.T) { &sqlstats.IteratorOptions{}, func( ctx context.Context, - statistics *roachpb.CollectedTransactionStatistics, + statistics *appstatspb.CollectedTransactionStatistics, ) error { if len(statistics.StatementFingerprintIDs) == 1 { if query, ok := stmtFingerprintIDToQueries[statistics.StatementFingerprintIDs[0]]; ok { @@ -211,12 +211,12 @@ func verifyStoredStmtFingerprints( ) { foundQueries := make(map[string]struct{}) foundTxns := make(map[string]struct{}) - stmtFingerprintIDToQueries := make(map[roachpb.StmtFingerprintID]string) + stmtFingerprintIDToQueries := make(map[appstatspb.StmtFingerprintID]string) require.NoError(t, sqlStats.IterateStatementStats( context.Background(), &sqlstats.IteratorOptions{}, - func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { if expectedExecCount, ok := expectedStmtFingerprints[statistics.Key.Query]; ok { foundQueries[statistics.Key.Query] = struct{}{} stmtFingerprintIDToQueries[statistics.ID] = statistics.Key.Query @@ -231,7 +231,7 @@ func verifyStoredStmtFingerprints( &sqlstats.IteratorOptions{}, func( ctx context.Context, - statistics *roachpb.CollectedTransactionStatistics, + statistics *appstatspb.CollectedTransactionStatistics, ) error { if len(statistics.StatementFingerprintIDs) == 1 { if query, ok := stmtFingerprintIDToQueries[statistics.StatementFingerprintIDs[0]]; ok { diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/BUILD.bazel b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/BUILD.bazel index aba5e8ab2ac0..a281f6430252 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/BUILD.bazel +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/BUILD.bazel @@ -13,7 +13,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", + "//pkg/sql/appstatspb", "//pkg/sql/sem/tree", "//pkg/util/encoding", "//pkg/util/json", @@ -32,7 +32,7 @@ go_test( args = ["-test.timeout=295s"], embed = [":sqlstatsutil"], deps = [ - "//pkg/roachpb", + "//pkg/sql/appstatspb", "//pkg/util/json", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_decoding.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_decoding.go index 38c6de9959ec..50b1035631c1 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_decoding.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_decoding.go @@ -11,16 +11,16 @@ package sqlstatsutil import ( - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) // DecodeTxnStatsMetadataJSON decodes the 'metadata' field of the JSON // representation of transaction statistics into -// roachpb.CollectedTransactionStatistics. +// appstatspb.CollectedTransactionStatistics. func DecodeTxnStatsMetadataJSON( - metadata json.JSON, result *roachpb.CollectedTransactionStatistics, + metadata json.JSON, result *appstatspb.CollectedTransactionStatistics, ) error { return jsonFields{ {"stmtFingerprintIDs", (*stmtFingerprintIDArray)(&result.StatementFingerprintIDs)}, @@ -28,39 +28,43 @@ func DecodeTxnStatsMetadataJSON( } // DecodeTxnStatsStatisticsJSON decodes the 'statistics' section of the -// transaction statistics JSON payload into roachpb.TransactionStatistics +// transaction statistics JSON payload into appstatspb.TransactionStatistics // protobuf. -func DecodeTxnStatsStatisticsJSON(jsonVal json.JSON, result *roachpb.TransactionStatistics) error { +func DecodeTxnStatsStatisticsJSON( + jsonVal json.JSON, result *appstatspb.TransactionStatistics, +) error { return (*txnStats)(result).decodeJSON(jsonVal) } // DecodeStmtStatsMetadataJSON decodes the 'metadata' field of the JSON // representation of the statement statistics into -// roachpb.CollectedStatementStatistics. +// appstatspb.CollectedStatementStatistics. func DecodeStmtStatsMetadataJSON( - metadata json.JSON, result *roachpb.CollectedStatementStatistics, + metadata json.JSON, result *appstatspb.CollectedStatementStatistics, ) error { return (*stmtStatsMetadata)(result).jsonFields().decodeJSON(metadata) } -// DecodeAggregatedMetadataJSON decodes the 'aggregated metadata' represented by roachpb.AggregatedStatementMetadata. +// DecodeAggregatedMetadataJSON decodes the 'aggregated metadata' represented by appstatspb.AggregatedStatementMetadata. func DecodeAggregatedMetadataJSON( - metadata json.JSON, result *roachpb.AggregatedStatementMetadata, + metadata json.JSON, result *appstatspb.AggregatedStatementMetadata, ) error { return (*aggregatedMetadata)(result).jsonFields().decodeJSON(metadata) } // DecodeStmtStatsStatisticsJSON decodes the 'statistics' field and the // 'execution_statistics' field in the given json into -// roachpb.StatementStatistics. -func DecodeStmtStatsStatisticsJSON(jsonVal json.JSON, result *roachpb.StatementStatistics) error { +// appstatspb.StatementStatistics. +func DecodeStmtStatsStatisticsJSON( + jsonVal json.JSON, result *appstatspb.StatementStatistics, +) error { return (*stmtStats)(result).decodeJSON(jsonVal) } // JSONToExplainTreePlanNode decodes the JSON-formatted ExplainTreePlanNode // produced by ExplainTreePlanNodeToJSON. -func JSONToExplainTreePlanNode(jsonVal json.JSON) (*roachpb.ExplainTreePlanNode, error) { - node := roachpb.ExplainTreePlanNode{} +func JSONToExplainTreePlanNode(jsonVal json.JSON) (*appstatspb.ExplainTreePlanNode, error) { + node := appstatspb.ExplainTreePlanNode{} nameAttr, err := jsonVal.FetchValKey("Name") if err != nil { @@ -120,7 +124,7 @@ func JSONToExplainTreePlanNode(jsonVal json.JSON) (*roachpb.ExplainTreePlanNode, if str != nil { value = *str } - node.Attrs = append(node.Attrs, &roachpb.ExplainTreePlanNode_Attr{ + node.Attrs = append(node.Attrs, &appstatspb.ExplainTreePlanNode_Attr{ Key: key, Value: value, }) diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go index b9eeae6625c2..e6522609cd16 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go @@ -13,7 +13,7 @@ package sqlstatsutil import ( "encoding/hex" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "golang.org/x/text/cases" @@ -21,7 +21,7 @@ import ( ) // ExplainTreePlanNodeToJSON builds a formatted JSON object from the explain tree nodes. -func ExplainTreePlanNodeToJSON(node *roachpb.ExplainTreePlanNode) json.JSON { +func ExplainTreePlanNodeToJSON(node *appstatspb.ExplainTreePlanNode) json.JSON { // Create a new json.ObjectBuilder with key-value pairs for the node's name (1), // node's attributes (len(node.Attrs)), and the node's children (1). @@ -43,7 +43,7 @@ func ExplainTreePlanNodeToJSON(node *roachpb.ExplainTreePlanNode) json.JSON { } // BuildStmtMetadataJSON returns a json.JSON object for the metadata section of -// the roachpb.CollectedStatementStatistics. +// the appstatspb.CollectedStatementStatistics. // JSON Schema for statement metadata: // // { @@ -61,12 +61,12 @@ func ExplainTreePlanNodeToJSON(node *roachpb.ExplainTreePlanNode) json.JSON { // "fullScan": { "type": "boolean" }, // } // } -func BuildStmtMetadataJSON(statistics *roachpb.CollectedStatementStatistics) (json.JSON, error) { +func BuildStmtMetadataJSON(statistics *appstatspb.CollectedStatementStatistics) (json.JSON, error) { return (*stmtStatsMetadata)(statistics).jsonFields().encodeJSON() } // BuildStmtStatisticsJSON encodes the statistics section a given -// roachpb.CollectedStatementStatistics into a json.JSON object. +// appstatspb.CollectedStatementStatistics into a json.JSON object. // // JSON Schema for stats portion: // @@ -159,12 +159,12 @@ func BuildStmtMetadataJSON(statistics *roachpb.CollectedStatementStatistics) (js // } // } // } -func BuildStmtStatisticsJSON(statistics *roachpb.StatementStatistics) (json.JSON, error) { +func BuildStmtStatisticsJSON(statistics *appstatspb.StatementStatistics) (json.JSON, error) { return (*stmtStats)(statistics).encodeJSON() } // BuildTxnMetadataJSON encodes the metadata portion a given -// roachpb.CollectedTransactionStatistics into a json.JSON object. +// appstatspb.CollectedTransactionStatistics into a json.JSON object. // // JSON Schema: // @@ -185,14 +185,16 @@ func BuildStmtStatisticsJSON(statistics *roachpb.StatementStatistics) (json.JSON // } // // TODO(azhng): add `firstExecAt` and `lastExecAt` into the protobuf definition. -func BuildTxnMetadataJSON(statistics *roachpb.CollectedTransactionStatistics) (json.JSON, error) { +func BuildTxnMetadataJSON( + statistics *appstatspb.CollectedTransactionStatistics, +) (json.JSON, error) { return jsonFields{ {"stmtFingerprintIDs", (*stmtFingerprintIDArray)(&statistics.StatementFingerprintIDs)}, }.encodeJSON() } // BuildTxnStatisticsJSON encodes the statistics portion a given -// roachpb.CollectedTransactionStatistics into a json.JSON. +// appstatspb.CollectedTransactionStatistics into a json.JSON. // // JSON Schema // @@ -261,12 +263,14 @@ func BuildTxnMetadataJSON(statistics *roachpb.CollectedTransactionStatistics) (j // } // } // } -func BuildTxnStatisticsJSON(statistics *roachpb.CollectedTransactionStatistics) (json.JSON, error) { +func BuildTxnStatisticsJSON( + statistics *appstatspb.CollectedTransactionStatistics, +) (json.JSON, error) { return (*txnStats)(&statistics.Stats).encodeJSON() } // BuildStmtDetailsMetadataJSON returns a json.JSON object for the aggregated metadata -// roachpb.AggregatedStatementMetadata. +// appstatspb.AggregatedStatementMetadata. // JSON Schema for statement aggregated metadata: // // { @@ -301,7 +305,7 @@ func BuildTxnStatisticsJSON(statistics *roachpb.CollectedTransactionStatistics) // } // } func BuildStmtDetailsMetadataJSON( - metadata *roachpb.AggregatedStatementMetadata, + metadata *appstatspb.AggregatedStatementMetadata, ) (json.JSON, error) { return (*aggregatedMetadata)(metadata).jsonFields().encodeJSON() } @@ -333,6 +337,6 @@ func EncodeUint64ToBytes(id uint64) []byte { return encoding.EncodeUint64Ascending(result, id) } -func encodeStmtFingerprintIDToString(id roachpb.StmtFingerprintID) string { +func encodeStmtFingerprintIDToString(id appstatspb.StmtFingerprintID) string { return hex.EncodeToString(EncodeUint64ToBytes(uint64(id))) } diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go index e49bc860e7c2..36fb56c7b4e2 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go @@ -16,7 +16,7 @@ import ( "strings" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -37,7 +37,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { t.Run("statement_statistics", func(t *testing.T) { data := genRandomData() - input := roachpb.CollectedStatementStatistics{} + input := appstatspb.CollectedStatementStatistics{} expectedMetadataStrTemplate := ` { @@ -148,7 +148,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { jsonTestHelper(t, expectedStatisticsStr, actualStatisticsJSON) // Ensure that we get the same protobuf after we decode the JSON. - var actualJSONUnmarshalled roachpb.CollectedStatementStatistics + var actualJSONUnmarshalled appstatspb.CollectedStatementStatistics err = DecodeStmtStatsMetadataJSON(actualMetadataJSON, &actualJSONUnmarshalled) require.NoError(t, err) @@ -166,7 +166,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { // the new one will be empty, without breaking the decoding process. t.Run("statement_statistics with new parameter", func(t *testing.T) { data := genRandomData() - expectedStatistics := roachpb.CollectedStatementStatistics{} + expectedStatistics := appstatspb.CollectedStatementStatistics{} expectedMetadataStrTemplate := ` { @@ -267,7 +267,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { actualStatisticsJSON, err := BuildStmtStatisticsJSON(&expectedStatistics.Stats) require.NoError(t, err) - var actualJSONUnmarshalled roachpb.CollectedStatementStatistics + var actualJSONUnmarshalled appstatspb.CollectedStatementStatistics err = DecodeStmtStatsMetadataJSON(actualMetadataJSON, &actualJSONUnmarshalled) require.NoError(t, err) @@ -276,7 +276,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { // the final actualJSONUnmarshalled.Stats. actualStatisticsJSON, _, _ = actualStatisticsJSON.RemovePath([]string{"statistics", "numRows"}) // Initialize the field again to remove the existing value. - expectedStatistics.Stats.NumRows = roachpb.NumericStat{} + expectedStatistics.Stats.NumRows = appstatspb.NumericStat{} // Strip the monononic part of timestamps, as it doesn't roundtrip. UTC() // has that stripping side-effect. expectedStatistics.Stats.LastExecTimestamp = expectedStatistics.Stats.LastExecTimestamp.UTC() @@ -289,8 +289,8 @@ func TestSQLStatsJsonEncoding(t *testing.T) { t.Run("transaction_statistics", func(t *testing.T) { data := genRandomData() - input := roachpb.CollectedTransactionStatistics{ - StatementFingerprintIDs: []roachpb.StmtFingerprintID{ + input := appstatspb.CollectedTransactionStatistics{ + StatementFingerprintIDs: []appstatspb.StmtFingerprintID{ 1, 100, 1000, 5467890, }, } @@ -384,7 +384,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { jsonTestHelper(t, expectedStatisticsStr, actualStatisticsJSON) // Ensure that we get the same protobuf after we decode the JSON. - var actualJSONUnmarshalled roachpb.CollectedTransactionStatistics + var actualJSONUnmarshalled appstatspb.CollectedTransactionStatistics err = DecodeTxnStatsMetadataJSON(actualMetadataJSON, &actualJSONUnmarshalled) require.NoError(t, err) @@ -397,7 +397,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { t.Run("statement aggregated metadata", func(t *testing.T) { data := genRandomData() - input := roachpb.AggregatedStatementMetadata{} + input := appstatspb.AggregatedStatementMetadata{} expectedAggregatedMetadataStrTemplate := ` { @@ -424,7 +424,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { jsonTestHelper(t, expectedAggregatedMetadataStr, actualMetadataJSON) // Ensure that we get the same protobuf after we decode the JSON. - var actualJSONUnmarshalled roachpb.AggregatedStatementMetadata + var actualJSONUnmarshalled appstatspb.AggregatedStatementMetadata err = DecodeAggregatedMetadataJSON(actualMetadataJSON, &actualJSONUnmarshalled) require.NoError(t, err) require.Equal(t, input, actualJSONUnmarshalled) @@ -434,7 +434,7 @@ func TestSQLStatsJsonEncoding(t *testing.T) { func BenchmarkSQLStatsJson(b *testing.B) { defer log.Scope(b).Close(b) b.Run("statement_stats", func(b *testing.B) { - inputStmtStats := roachpb.CollectedStatementStatistics{} + inputStmtStats := appstatspb.CollectedStatementStatistics{} b.Run("encoding", func(b *testing.B) { b.SetBytes(int64(inputStmtStats.Size())) @@ -452,7 +452,7 @@ func BenchmarkSQLStatsJson(b *testing.B) { inputStmtStatsMetaJSON, _ := BuildStmtMetadataJSON(&inputStmtStats) inputStmtStatsJSON, _ := BuildStmtStatisticsJSON(&inputStmtStats.Stats) - result := roachpb.CollectedStatementStatistics{} + result := appstatspb.CollectedStatementStatistics{} b.Run("decoding", func(b *testing.B) { b.SetBytes(int64(inputStmtStatsJSON.Size() + inputStmtStatsMetaJSON.Size())) @@ -471,7 +471,7 @@ func BenchmarkSQLStatsJson(b *testing.B) { }) b.Run("transaction_stats", func(b *testing.B) { - inputTxnStats := roachpb.CollectedTransactionStatistics{} + inputTxnStats := appstatspb.CollectedTransactionStatistics{} b.Run("encoding", func(b *testing.B) { b.SetBytes(int64(inputTxnStats.Size())) @@ -493,7 +493,7 @@ func BenchmarkSQLStatsJson(b *testing.B) { b.Fatal(err) } - result := roachpb.CollectedTransactionStatistics{} + result := appstatspb.CollectedTransactionStatistics{} b.Run("decoding", func(b *testing.B) { b.SetBytes(int64(inputTxnStatsJSON.Size() + inputTxnStatsMetaJSON.Size())) @@ -512,8 +512,8 @@ func BenchmarkSQLStatsJson(b *testing.B) { }) b.Run("statement_metadata", func(b *testing.B) { - inputStmtStats := roachpb.CollectedStatementStatistics{} - inputStmtMetadata := roachpb.AggregatedStatementMetadata{} + inputStmtStats := appstatspb.CollectedStatementStatistics{} + inputStmtMetadata := appstatspb.AggregatedStatementMetadata{} b.Run("encoding", func(b *testing.B) { b.SetBytes(int64(inputStmtStats.Size())) @@ -526,7 +526,7 @@ func BenchmarkSQLStatsJson(b *testing.B) { }) inputStmtStatsAggregatedMetaJSON, _ := BuildStmtMetadataJSON(&inputStmtStats) - result := roachpb.AggregatedStatementMetadata{} + result := appstatspb.AggregatedStatementMetadata{} b.Run("decoding", func(b *testing.B) { b.SetBytes(int64(inputStmtStatsAggregatedMetaJSON.Size())) @@ -547,32 +547,32 @@ func TestExplainTreePlanNodeToJSON(t *testing.T) { defer leaktest.AfterTest(t)() testDataArr := []struct { - explainTree roachpb.ExplainTreePlanNode + explainTree appstatspb.ExplainTreePlanNode expected string }{ // Test data using a node with multiple inner children. { - roachpb.ExplainTreePlanNode{ + appstatspb.ExplainTreePlanNode{ Name: "root", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "rootKey", Value: "rootValue", }, }, - Children: []*roachpb.ExplainTreePlanNode{ + Children: []*appstatspb.ExplainTreePlanNode{ { Name: "child", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "childKey", Value: "childValue", }, }, - Children: []*roachpb.ExplainTreePlanNode{ + Children: []*appstatspb.ExplainTreePlanNode{ { Name: "innerChild", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "innerChildKey", Value: "innerChildValue", @@ -587,9 +587,9 @@ func TestExplainTreePlanNodeToJSON(t *testing.T) { }, // Test using a node with multiple attributes. { - roachpb.ExplainTreePlanNode{ + appstatspb.ExplainTreePlanNode{ Name: "root", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "rootFirstKey", Value: "rootFirstValue", @@ -599,10 +599,10 @@ func TestExplainTreePlanNodeToJSON(t *testing.T) { Value: "rootSecondValue", }, }, - Children: []*roachpb.ExplainTreePlanNode{ + Children: []*appstatspb.ExplainTreePlanNode{ { Name: "child", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "childKey", Value: "childValue", @@ -615,27 +615,27 @@ func TestExplainTreePlanNodeToJSON(t *testing.T) { }, // Test using a node with multiple children and multiple inner children. { - roachpb.ExplainTreePlanNode{ + appstatspb.ExplainTreePlanNode{ Name: "root", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "rootKey", Value: "rootValue", }, }, - Children: []*roachpb.ExplainTreePlanNode{ + Children: []*appstatspb.ExplainTreePlanNode{ { Name: "firstChild", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "firstChildKey", Value: "firstChildValue", }, }, - Children: []*roachpb.ExplainTreePlanNode{ + Children: []*appstatspb.ExplainTreePlanNode{ { Name: "innerChild", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "innerChildKey", Value: "innerChildValue", @@ -646,7 +646,7 @@ func TestExplainTreePlanNodeToJSON(t *testing.T) { }, { Name: "secondChild", - Attrs: []*roachpb.ExplainTreePlanNode_Attr{ + Attrs: []*appstatspb.ExplainTreePlanNode_Attr{ { Key: "secondChildKey", Value: "secondChildValue", @@ -669,7 +669,7 @@ func TestExplainTreePlanNodeToJSON(t *testing.T) { } } -type nodeAttrList []*roachpb.ExplainTreePlanNode_Attr +type nodeAttrList []*appstatspb.ExplainTreePlanNode_Attr var _ sort.Interface = nodeAttrList{} @@ -687,7 +687,7 @@ func (n nodeAttrList) Swap(i, j int) { n[j] = tmp } -type nodeList []*roachpb.ExplainTreePlanNode +type nodeList []*appstatspb.ExplainTreePlanNode var _ sort.Interface = nodeList{} @@ -705,7 +705,7 @@ func (n nodeList) Swap(i, j int) { n[j] = tmp } -func compareExplainTree(t *testing.T, expected, actual *roachpb.ExplainTreePlanNode) { +func compareExplainTree(t *testing.T, expected, actual *appstatspb.ExplainTreePlanNode) { require.Equal(t, strings.ToLower(expected.Name), strings.ToLower(actual.Name)) require.Equal(t, len(expected.Attrs), len(actual.Attrs)) diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go index 1d05fa864e54..bcd184d3ffdb 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go @@ -15,7 +15,7 @@ import ( "time" "github.com/cockroachdb/apd/v3" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" @@ -52,7 +52,7 @@ var ( _ jsonMarshaler = (*int64Array)(nil) ) -type txnStats roachpb.TransactionStatistics +type txnStats appstatspb.TransactionStatistics func (t *txnStats) jsonFields() jsonFields { return jsonFields{ @@ -69,7 +69,7 @@ func (t *txnStats) encodeJSON() (json.JSON, error) { return t.jsonFields().encodeJSON() } -type stmtStats roachpb.StatementStatistics +type stmtStats appstatspb.StatementStatistics func (s *stmtStats) jsonFields() jsonFields { return jsonFields{ @@ -87,7 +87,7 @@ func (s *stmtStats) encodeJSON() (json.JSON, error) { return s.jsonFields().encodeJSON() } -type stmtStatsMetadata roachpb.CollectedStatementStatistics +type stmtStatsMetadata appstatspb.CollectedStatementStatistics func (s *stmtStatsMetadata) jsonFields() jsonFields { return jsonFields{ @@ -103,7 +103,7 @@ func (s *stmtStatsMetadata) jsonFields() jsonFields { } } -type aggregatedMetadata roachpb.AggregatedStatementMetadata +type aggregatedMetadata appstatspb.AggregatedStatementMetadata func (s *aggregatedMetadata) jsonFields() jsonFields { return jsonFields{ @@ -233,7 +233,7 @@ func (a *stringArray) encodeJSON() (json.JSON, error) { return builder.Build(), nil } -type stmtFingerprintIDArray []roachpb.StmtFingerprintID +type stmtFingerprintIDArray []appstatspb.StmtFingerprintID func (s *stmtFingerprintIDArray) decodeJSON(js json.JSON) error { arrLen := js.Len() @@ -246,7 +246,7 @@ func (s *stmtFingerprintIDArray) decodeJSON(js json.JSON) error { if err := fingerprintID.decodeJSON(fingerprintIDJSON); err != nil { return err } - *s = append(*s, roachpb.StmtFingerprintID(fingerprintID)) + *s = append(*s, appstatspb.StmtFingerprintID(fingerprintID)) } return nil @@ -266,7 +266,7 @@ func (s *stmtFingerprintIDArray) encodeJSON() (json.JSON, error) { return builder.Build(), nil } -type stmtFingerprintID roachpb.StmtFingerprintID +type stmtFingerprintID appstatspb.StmtFingerprintID func (s *stmtFingerprintID) decodeJSON(js json.JSON) error { var str jsonString @@ -290,10 +290,10 @@ func (s *stmtFingerprintID) decodeJSON(js json.JSON) error { func (s *stmtFingerprintID) encodeJSON() (json.JSON, error) { return json.FromString( - encodeStmtFingerprintIDToString((roachpb.StmtFingerprintID)(*s))), nil + encodeStmtFingerprintIDToString((appstatspb.StmtFingerprintID)(*s))), nil } -type innerTxnStats roachpb.TransactionStatistics +type innerTxnStats appstatspb.TransactionStatistics func (t *innerTxnStats) jsonFields() jsonFields { return jsonFields{ @@ -318,7 +318,7 @@ func (t *innerTxnStats) encodeJSON() (json.JSON, error) { return t.jsonFields().encodeJSON() } -type innerStmtStats roachpb.StatementStatistics +type innerStmtStats appstatspb.StatementStatistics func (s *innerStmtStats) jsonFields() jsonFields { return jsonFields{ @@ -350,7 +350,7 @@ func (s *innerStmtStats) encodeJSON() (json.JSON, error) { return s.jsonFields().encodeJSON() } -type execStats roachpb.ExecStats +type execStats appstatspb.ExecStats func (e *execStats) jsonFields() jsonFields { return jsonFields{ @@ -372,7 +372,7 @@ func (e *execStats) encodeJSON() (json.JSON, error) { return e.jsonFields().encodeJSON() } -type numericStats roachpb.NumericStat +type numericStats appstatspb.NumericStat func (n *numericStats) jsonFields() jsonFields { return jsonFields{ diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/testutils.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/testutils.go index 41e65352c013..574cc19159cb 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/testutils.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/testutils.go @@ -20,16 +20,16 @@ import ( "text/template" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/stretchr/testify/require" ) // GetRandomizedCollectedStatementStatisticsForTest returns a -// roachpb.CollectedStatementStatistics with its fields randomly filled. +// appstatspb.CollectedStatementStatistics with its fields randomly filled. func GetRandomizedCollectedStatementStatisticsForTest( t *testing.T, -) (result roachpb.CollectedStatementStatistics) { +) (result appstatspb.CollectedStatementStatistics) { data := genRandomData() fillObject(t, reflect.ValueOf(&result), &data) diff --git a/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go b/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go index 2f4ba4dcb7e3..84b6b7b27e37 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go +++ b/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go @@ -17,7 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -149,22 +149,22 @@ FROM return query, len(selectedColumns) } -func rowToStmtStats(row tree.Datums) (*roachpb.CollectedStatementStatistics, error) { - var stats roachpb.CollectedStatementStatistics +func rowToStmtStats(row tree.Datums) (*appstatspb.CollectedStatementStatistics, error) { + var stats appstatspb.CollectedStatementStatistics stats.AggregatedTs = tree.MustBeDTimestampTZ(row[0]).Time stmtFingerprintID, err := sqlstatsutil.DatumToUint64(row[1]) if err != nil { return nil, err } - stats.ID = roachpb.StmtFingerprintID(stmtFingerprintID) + stats.ID = appstatspb.StmtFingerprintID(stmtFingerprintID) transactionFingerprintID, err := sqlstatsutil.DatumToUint64(row[2]) if err != nil { return nil, err } stats.Key.TransactionFingerprintID = - roachpb.TransactionFingerprintID(transactionFingerprintID) + appstatspb.TransactionFingerprintID(transactionFingerprintID) stats.Key.PlanHash, err = sqlstatsutil.DatumToUint64(row[3]) if err != nil { diff --git a/pkg/sql/sqlstats/persistedsqlstats/txn_reader.go b/pkg/sql/sqlstats/persistedsqlstats/txn_reader.go index 517c509fe021..71b6921c3ea9 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/txn_reader.go +++ b/pkg/sql/sqlstats/persistedsqlstats/txn_reader.go @@ -16,7 +16,7 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -134,8 +134,8 @@ FROM return query, len(selectedColumns) } -func rowToTxnStats(row tree.Datums) (*roachpb.CollectedTransactionStatistics, error) { - var stats roachpb.CollectedTransactionStatistics +func rowToTxnStats(row tree.Datums) (*appstatspb.CollectedTransactionStatistics, error) { + var stats appstatspb.CollectedTransactionStatistics var err error stats.AggregatedTs = tree.MustBeDTimestampTZ(row[0]).Time @@ -144,7 +144,7 @@ func rowToTxnStats(row tree.Datums) (*roachpb.CollectedTransactionStatistics, er if err != nil { return nil, err } - stats.TransactionFingerprintID = roachpb.TransactionFingerprintID(value) + stats.TransactionFingerprintID = appstatspb.TransactionFingerprintID(value) stats.App = string(tree.MustBeDString(row[2])) diff --git a/pkg/sql/sqlstats/sslocal/BUILD.bazel b/pkg/sql/sqlstats/sslocal/BUILD.bazel index 8d87eed24555..0ffe4a3371b8 100644 --- a/pkg/sql/sqlstats/sslocal/BUILD.bazel +++ b/pkg/sql/sqlstats/sslocal/BUILD.bazel @@ -16,10 +16,10 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", "//pkg/server/serverpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/sessionphase", "//pkg/sql/sqlstats", "//pkg/sql/sqlstats/insights", @@ -45,7 +45,6 @@ go_test( deps = [ ":sslocal", "//pkg/base", - "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/security/username", @@ -54,6 +53,7 @@ go_test( "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql", + "//pkg/sql/appstatspb", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", "//pkg/sql/sessionphase", diff --git a/pkg/sql/sqlstats/sslocal/iterator_test.go b/pkg/sql/sqlstats/sslocal/iterator_test.go index b23dcb4bf2fe..602cb7705c0a 100644 --- a/pkg/sql/sqlstats/sslocal/iterator_test.go +++ b/pkg/sql/sqlstats/sslocal/iterator_test.go @@ -14,8 +14,8 @@ import ( "context" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -49,10 +49,10 @@ func TestSQLStatsIteratorWithTelemetryFlush(t *testing.T) { // We collect all the statement fingerprint IDs so that we can test the // transaction stats later. - fingerprintIDs := make(map[roachpb.StmtFingerprintID]struct{}) + fingerprintIDs := make(map[appstatspb.StmtFingerprintID]struct{}) require.NoError(t, sqlStats.IterateStatementStats(ctx, &sqlstats.IteratorOptions{}, - func(_ context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(_ context.Context, statistics *appstatspb.CollectedStatementStatistics) error { fingerprintIDs[statistics.ID] = struct{}{} return nil })) @@ -62,7 +62,7 @@ func TestSQLStatsIteratorWithTelemetryFlush(t *testing.T) { sqlStats.IterateStatementStats( ctx, &sqlstats.IteratorOptions{}, - func(_ context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(_ context.Context, statistics *appstatspb.CollectedStatementStatistics) error { require.NotNil(t, statistics) // If we are running our test case, we reset the SQL Stats. The iterator // should gracefully handle that. @@ -83,7 +83,7 @@ func TestSQLStatsIteratorWithTelemetryFlush(t *testing.T) { &sqlstats.IteratorOptions{}, func( ctx context.Context, - statistics *roachpb.CollectedTransactionStatistics, + statistics *appstatspb.CollectedTransactionStatistics, ) error { require.NotNil(t, statistics) diff --git a/pkg/sql/sqlstats/sslocal/sql_stats_test.go b/pkg/sql/sqlstats/sslocal/sql_stats_test.go index c22905df2a28..dc73e5eb7a19 100644 --- a/pkg/sql/sqlstats/sslocal/sql_stats_test.go +++ b/pkg/sql/sqlstats/sslocal/sql_stats_test.go @@ -22,12 +22,12 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sessionphase" @@ -78,7 +78,7 @@ func TestStmtStatsBulkIngestWithRandomMetadata(t *testing.T) { &sqlstats.IteratorOptions{}, func( ctx context.Context, - statistics *roachpb.CollectedStatementStatistics, + statistics *appstatspb.CollectedStatementStatistics, ) error { var found bool for i := range testData { @@ -97,73 +97,73 @@ func TestSQLStatsStmtStatsBulkIngest(t *testing.T) { defer log.Scope(t).Close(t) testData := []struct { - id roachpb.StmtFingerprintID - key roachpb.StatementStatisticsKey - stats roachpb.StatementStatistics + id appstatspb.StmtFingerprintID + key appstatspb.StatementStatisticsKey + stats appstatspb.StatementStatistics }{ { id: 0, - key: roachpb.StatementStatisticsKey{ + key: appstatspb.StatementStatisticsKey{ App: "app1", Query: "SELECT 1", Database: "testdb", }, - stats: roachpb.StatementStatistics{ + stats: appstatspb.StatementStatistics{ Count: 7, }, }, { id: 0, - key: roachpb.StatementStatisticsKey{ + key: appstatspb.StatementStatisticsKey{ App: "app0", Query: "SELECT 1", Database: "testdb", }, - stats: roachpb.StatementStatistics{ + stats: appstatspb.StatementStatistics{ Count: 2, }, }, { id: 1, - key: roachpb.StatementStatisticsKey{ + key: appstatspb.StatementStatisticsKey{ App: "app100", Query: "SELECT 1,1", Database: "testdb", }, - stats: roachpb.StatementStatistics{ + stats: appstatspb.StatementStatistics{ Count: 31, }, }, { id: 1, - key: roachpb.StatementStatisticsKey{ + key: appstatspb.StatementStatisticsKey{ App: "app0", Query: "SELECT 1,1", Database: "testdb", }, - stats: roachpb.StatementStatistics{ + stats: appstatspb.StatementStatistics{ Count: 32, }, }, { id: 0, - key: roachpb.StatementStatisticsKey{ + key: appstatspb.StatementStatisticsKey{ App: "app1", Query: "SELECT 1", Database: "testdb", }, - stats: roachpb.StatementStatistics{ + stats: appstatspb.StatementStatistics{ Count: 33, }, }, { id: 1, - key: roachpb.StatementStatisticsKey{ + key: appstatspb.StatementStatisticsKey{ App: "app100", Query: "SELECT 1,1", Database: "testdb", }, - stats: roachpb.StatementStatistics{ + stats: appstatspb.StatementStatistics{ Count: 2, }, }, @@ -197,7 +197,7 @@ func TestSQLStatsStmtStatsBulkIngest(t *testing.T) { &sqlstats.IteratorOptions{}, func( ctx context.Context, - statistics *roachpb.CollectedStatementStatistics, + statistics *appstatspb.CollectedStatementStatistics, ) error { require.Equal(t, "testdb", statistics.Key.Database) foundStats[statistics.Key.App+statistics.Key.Query] = statistics.Stats.Count @@ -212,65 +212,65 @@ func TestSQLStatsTxnStatsBulkIngest(t *testing.T) { defer log.Scope(t).Close(t) testData := []struct { - stats roachpb.CollectedTransactionStatistics + stats appstatspb.CollectedTransactionStatistics }{ { - stats: roachpb.CollectedTransactionStatistics{ - TransactionFingerprintID: roachpb.TransactionFingerprintID(0), + stats: appstatspb.CollectedTransactionStatistics{ + TransactionFingerprintID: appstatspb.TransactionFingerprintID(0), App: "app1", - Stats: roachpb.TransactionStatistics{ + Stats: appstatspb.TransactionStatistics{ Count: 7, }, }, }, { - stats: roachpb.CollectedTransactionStatistics{ - TransactionFingerprintID: roachpb.TransactionFingerprintID(0), + stats: appstatspb.CollectedTransactionStatistics{ + TransactionFingerprintID: appstatspb.TransactionFingerprintID(0), App: "app0", - Stats: roachpb.TransactionStatistics{ + Stats: appstatspb.TransactionStatistics{ Count: 2, }, }, }, { - stats: roachpb.CollectedTransactionStatistics{ - TransactionFingerprintID: roachpb.TransactionFingerprintID(1), + stats: appstatspb.CollectedTransactionStatistics{ + TransactionFingerprintID: appstatspb.TransactionFingerprintID(1), App: "app100", - Stats: roachpb.TransactionStatistics{ + Stats: appstatspb.TransactionStatistics{ Count: 31, }, }, }, { - stats: roachpb.CollectedTransactionStatistics{ - TransactionFingerprintID: roachpb.TransactionFingerprintID(1), + stats: appstatspb.CollectedTransactionStatistics{ + TransactionFingerprintID: appstatspb.TransactionFingerprintID(1), App: "app0", - Stats: roachpb.TransactionStatistics{ + Stats: appstatspb.TransactionStatistics{ Count: 32, }, }, }, { - stats: roachpb.CollectedTransactionStatistics{ - TransactionFingerprintID: roachpb.TransactionFingerprintID(0), + stats: appstatspb.CollectedTransactionStatistics{ + TransactionFingerprintID: appstatspb.TransactionFingerprintID(0), App: "app1", - Stats: roachpb.TransactionStatistics{ + Stats: appstatspb.TransactionStatistics{ Count: 33, }, }, }, { - stats: roachpb.CollectedTransactionStatistics{ - TransactionFingerprintID: roachpb.TransactionFingerprintID(1), + stats: appstatspb.CollectedTransactionStatistics{ + TransactionFingerprintID: appstatspb.TransactionFingerprintID(1), App: "app100", - Stats: roachpb.TransactionStatistics{ + Stats: appstatspb.TransactionStatistics{ Count: 2, }, }, }, } - expectedCount := make(map[roachpb.TransactionFingerprintID]int64) + expectedCount := make(map[appstatspb.TransactionFingerprintID]int64) input := make([]serverpb.StatementsResponse_ExtendedCollectedTransactionStatistics, 0, len(testData)) @@ -288,14 +288,14 @@ func TestSQLStatsTxnStatsBulkIngest(t *testing.T) { sqlStats, err := sslocal.NewTempSQLStatsFromExistingTxnStats(input) require.NoError(t, err) - foundStats := make(map[roachpb.TransactionFingerprintID]int64) + foundStats := make(map[appstatspb.TransactionFingerprintID]int64) require.NoError(t, sqlStats.IterateTransactionStats( context.Background(), &sqlstats.IteratorOptions{}, func( ctx context.Context, - statistics *roachpb.CollectedTransactionStatistics, + statistics *appstatspb.CollectedTransactionStatistics, ) error { foundStats[statistics.TransactionFingerprintID] = statistics.Stats.Count return nil @@ -467,7 +467,7 @@ func TestExplicitTxnFingerprintAccounting(t *testing.T) { ) recordStats := func(testCase *tc) { - var txnFingerprintID roachpb.TransactionFingerprintID + var txnFingerprintID appstatspb.TransactionFingerprintID txnFingerprintIDHash := util.MakeFNV64() statsCollector.StartTransaction() defer func() { @@ -487,7 +487,7 @@ func TestExplicitTxnFingerprintAccounting(t *testing.T) { for _, fingerprint := range testCase.fingerprints { stmtFingerprintID, err := statsCollector.RecordStatement( ctx, - roachpb.StatementStatisticsKey{ + appstatspb.StatementStatisticsKey{ Query: fingerprint, ImplicitTxn: testCase.implicit, }, @@ -496,7 +496,7 @@ func TestExplicitTxnFingerprintAccounting(t *testing.T) { require.NoError(t, err) txnFingerprintIDHash.Add(uint64(stmtFingerprintID)) } - txnFingerprintID = roachpb.TransactionFingerprintID(txnFingerprintIDHash.Sum()) + txnFingerprintID = appstatspb.TransactionFingerprintID(txnFingerprintIDHash.Sum()) } for _, tc := range testCases { @@ -591,14 +591,14 @@ func TestAssociatingStmtStatsWithTxnFingerprint(t *testing.T) { for _, fingerprint := range txn.stmtFingerprints { stmtFingerprintID, err := statsCollector.RecordStatement( ctx, - roachpb.StatementStatisticsKey{Query: fingerprint}, + appstatspb.StatementStatisticsKey{Query: fingerprint}, sqlstats.RecordedStmtStats{}, ) require.NoError(t, err) txnFingerprintIDHash.Add(uint64(stmtFingerprintID)) } - transactionFingerprintID := roachpb.TransactionFingerprintID(txnFingerprintIDHash.Sum()) + transactionFingerprintID := appstatspb.TransactionFingerprintID(txnFingerprintIDHash.Sum()) statsCollector.EndTransaction(ctx, transactionFingerprintID) err := statsCollector.RecordTransaction(ctx, transactionFingerprintID, sqlstats.RecordedTxnStats{ SessionData: &sessiondata.SessionData{ @@ -612,11 +612,11 @@ func TestAssociatingStmtStatsWithTxnFingerprint(t *testing.T) { require.NoError(t, err) // Gather the collected stats so that we can assert on them. - var stats []*roachpb.CollectedStatementStatistics + var stats []*appstatspb.CollectedStatementStatistics err = statsCollector.IterateStatementStats( ctx, &sqlstats.IteratorOptions{}, - func(_ context.Context, s *roachpb.CollectedStatementStatistics) error { + func(_ context.Context, s *appstatspb.CollectedStatementStatistics) error { stats = append(stats, s) return nil }, diff --git a/pkg/sql/sqlstats/sslocal/sslocal_iterator.go b/pkg/sql/sqlstats/sslocal/sslocal_iterator.go index 36efa51b0432..c704c5d56095 100644 --- a/pkg/sql/sqlstats/sslocal/sslocal_iterator.go +++ b/pkg/sql/sqlstats/sslocal/sslocal_iterator.go @@ -11,7 +11,7 @@ package sslocal import ( - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/ssmemstorage" ) @@ -68,9 +68,9 @@ func (s *StmtStatsIterator) Next() bool { return true } -// Cur returns the roachpb.CollectedStatementStatistics at the current internal +// Cur returns the appstatspb.CollectedStatementStatistics at the current internal // counter. -func (s *StmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { +func (s *StmtStatsIterator) Cur() *appstatspb.CollectedStatementStatistics { return s.curIter.Cur() } @@ -119,8 +119,8 @@ func (t *TxnStatsIterator) Next() bool { return true } -// Cur returns the roachpb.CollectedTransactionStatistics at the current internal +// Cur returns the appstatspb.CollectedTransactionStatistics at the current internal // counter. -func (t *TxnStatsIterator) Cur() *roachpb.CollectedTransactionStatistics { +func (t *TxnStatsIterator) Cur() *appstatspb.CollectedTransactionStatistics { return t.curIter.Cur() } diff --git a/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go b/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go index 89655683f276..0acc363ddd2a 100644 --- a/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go +++ b/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go @@ -13,8 +13,8 @@ package sslocal import ( "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sessionphase" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -82,21 +82,21 @@ func (s *StatsCollector) StartTransaction() { // EndTransaction implements sqlstats.StatsCollector interface. func (s *StatsCollector) EndTransaction( - ctx context.Context, transactionFingerprintID roachpb.TransactionFingerprintID, + ctx context.Context, transactionFingerprintID appstatspb.TransactionFingerprintID, ) { // We possibly ignore the transactionFingerprintID, for situations where // grouping by it would otherwise result in collecting higher-cardinality // data in the system tables than the cleanup job is able to keep up with. // See #78338. if !AssociateStmtWithTxnFingerprint.Get(&s.st.SV) { - transactionFingerprintID = roachpb.InvalidTransactionFingerprintID + transactionFingerprintID = appstatspb.InvalidTransactionFingerprintID } var discardedStats uint64 discardedStats += s.flushTarget.MergeApplicationStatementStats( ctx, s.ApplicationStats, - func(statistics *roachpb.CollectedStatementStatistics) { + func(statistics *appstatspb.CollectedStatementStatistics) { statistics.Key.TransactionFingerprintID = transactionFingerprintID }, ) @@ -135,7 +135,7 @@ func (s *StatsCollector) ShouldSample( // UpgradeImplicitTxn implements sqlstats.StatsCollector interface. func (s *StatsCollector) UpgradeImplicitTxn(ctx context.Context) error { err := s.ApplicationStats.IterateStatementStats(ctx, &sqlstats.IteratorOptions{}, - func(_ context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(_ context.Context, statistics *appstatspb.CollectedStatementStatistics) error { statistics.Key.ImplicitTxn = false return nil }) diff --git a/pkg/sql/sqlstats/sslocal/temp_sql_stats.go b/pkg/sql/sqlstats/sslocal/temp_sql_stats.go index a952d2f7e860..08238a1687f8 100644 --- a/pkg/sql/sqlstats/sslocal/temp_sql_stats.go +++ b/pkg/sql/sqlstats/sslocal/temp_sql_stats.go @@ -56,7 +56,7 @@ func (t txnResponseList) Swap(i, j int) { } // NewTempSQLStatsFromExistingStmtStats returns an instance of SQLStats populated -// from the provided slice of roachpb.CollectedStatementStatistics. +// from the provided slice of appstatspb.CollectedStatementStatistics. // // This constructor returns a variant of SQLStats which is used to aggregate // RPC-fanout results. This means that, unliked the regular SQLStats, whose diff --git a/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel b/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel index a115c50818aa..6ab7ea1d48e3 100644 --- a/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel +++ b/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/server/serverpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/appstatspb", "//pkg/sql/execstats", "//pkg/sql/sqlstats", "//pkg/sql/sqlstats/insights", diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go index 8ad677314bb4..d50715209daf 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go @@ -13,7 +13,7 @@ package ssmemstorage import ( "sort" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" ) @@ -27,7 +27,7 @@ type baseIterator struct { type StmtStatsIterator struct { baseIterator stmtKeys stmtList - currentValue *roachpb.CollectedStatementStatistics + currentValue *appstatspb.CollectedStatementStatistics } // NewStmtStatsIterator returns a StmtStatsIterator. @@ -83,8 +83,8 @@ func (s *StmtStatsIterator) Next() bool { querySummary := statementStats.mu.querySummary statementStats.mu.Unlock() - s.currentValue = &roachpb.CollectedStatementStatistics{ - Key: roachpb.StatementStatisticsKey{ + s.currentValue = &appstatspb.CollectedStatementStatistics{ + Key: appstatspb.StatementStatisticsKey{ Query: stmtKey.stmtNoConstants, QuerySummary: querySummary, DistSQL: distSQLUsed, @@ -104,9 +104,9 @@ func (s *StmtStatsIterator) Next() bool { return true } -// Cur returns the roachpb.CollectedStatementStatistics at the current internal +// Cur returns the appstatspb.CollectedStatementStatistics at the current internal // counter. -func (s *StmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { +func (s *StmtStatsIterator) Cur() *appstatspb.CollectedStatementStatistics { return s.currentValue } @@ -115,7 +115,7 @@ func (s *StmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { type TxnStatsIterator struct { baseIterator txnKeys txnList - curValue *roachpb.CollectedTransactionStatistics + curValue *appstatspb.CollectedTransactionStatistics } // NewTxnStatsIterator returns a new instance of TxnStatsIterator. @@ -166,7 +166,7 @@ func (t *TxnStatsIterator) Next() bool { txnStats.mu.Lock() defer txnStats.mu.Unlock() - t.curValue = &roachpb.CollectedTransactionStatistics{ + t.curValue = &appstatspb.CollectedTransactionStatistics{ StatementFingerprintIDs: txnStats.statementFingerprintIDs, App: t.container.appName, Stats: txnStats.mu.data, @@ -176,8 +176,8 @@ func (t *TxnStatsIterator) Next() bool { return true } -// Cur returns the roachpb.CollectedTransactionStatistics at the current internal +// Cur returns the appstatspb.CollectedTransactionStatistics at the current internal // counter. -func (t *TxnStatsIterator) Cur() *roachpb.CollectedTransactionStatistics { +func (t *TxnStatsIterator) Cur() *appstatspb.CollectedTransactionStatistics { return t.curValue } diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go index 548a49bc99fb..4a701b10ea28 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go @@ -22,10 +22,10 @@ import ( "time" "unsafe" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/execstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights" @@ -43,7 +43,7 @@ import ( type stmtKey struct { sampledPlanKey planHash uint64 - transactionFingerprintID roachpb.TransactionFingerprintID + transactionFingerprintID appstatspb.TransactionFingerprintID } // sampledPlanKey is used by the Optimizer to determine if we should build a full EXPLAIN plan. @@ -106,7 +106,7 @@ type Container struct { acc mon.BoundAccount stmts map[stmtKey]*stmtStats - txns map[roachpb.TransactionFingerprintID]*txnStats + txns map[appstatspb.TransactionFingerprintID]*txnStats // sampledPlanMetadataCache records when was the last time the plan was // sampled. This data structure uses a subset of stmtKey as the key into @@ -152,7 +152,7 @@ func New( } s.mu.stmts = make(map[stmtKey]*stmtStats) - s.mu.txns = make(map[roachpb.TransactionFingerprintID]*txnStats) + s.mu.txns = make(map[appstatspb.TransactionFingerprintID]*txnStats) s.mu.sampledPlanMetadataCache = make(map[sampledPlanKey]time.Time) s.atomic.uniqueStmtFingerprintCount = uniqueStmtFingerprintCount @@ -166,7 +166,7 @@ func New( func (s *Container) IterateAggregatedTransactionStats( _ context.Context, _ *sqlstats.IteratorOptions, visitor sqlstats.AggregatedTransactionVisitor, ) error { - txnStat := func() roachpb.TxnStats { + txnStat := func() appstatspb.TxnStats { s.txnCounts.mu.Lock() defer s.txnCounts.mu.Unlock() return s.txnCounts.mu.TxnStats @@ -369,29 +369,29 @@ func (s *Container) NewApplicationStatsWithInheritedOptions() sqlstats.Applicati } type txnStats struct { - statementFingerprintIDs []roachpb.StmtFingerprintID + statementFingerprintIDs []appstatspb.StmtFingerprintID mu struct { syncutil.Mutex - data roachpb.TransactionStatistics + data appstatspb.TransactionStatistics } } func (t *txnStats) sizeUnsafe() int64 { const txnStatsShallowSize = int64(unsafe.Sizeof(txnStats{})) stmtFingerprintIDsSize := int64(cap(t.statementFingerprintIDs)) * - int64(unsafe.Sizeof(roachpb.StmtFingerprintID(0))) + int64(unsafe.Sizeof(appstatspb.StmtFingerprintID(0))) // t.mu.data might contain pointer types, so we subtract its shallow size // and include the actual size. - dataSize := -int64(unsafe.Sizeof(roachpb.TransactionStatistics{})) + + dataSize := -int64(unsafe.Sizeof(appstatspb.TransactionStatistics{})) + int64(t.mu.data.Size()) return txnStatsShallowSize + stmtFingerprintIDsSize + dataSize } -func (t *txnStats) mergeStats(stats *roachpb.TransactionStatistics) { +func (t *txnStats) mergeStats(stats *appstatspb.TransactionStatistics) { t.mu.Lock() defer t.mu.Unlock() t.mu.data.Add(stats) @@ -400,7 +400,7 @@ func (t *txnStats) mergeStats(stats *roachpb.TransactionStatistics) { // stmtStats holds per-statement statistics. type stmtStats struct { // ID is the statementFingerprintID constructed using the stmtKey fields. - ID roachpb.StmtFingerprintID + ID appstatspb.StmtFingerprintID // data contains all fields that are modified when new statements matching // the stmtKey are executed, and therefore must be protected by a mutex. @@ -426,7 +426,7 @@ type stmtStats struct { // querySummary records a summarized format of the query statement. querySummary string - data roachpb.StatementStatistics + data appstatspb.StatementStatistics } } @@ -436,7 +436,7 @@ func (s *stmtStats) sizeUnsafe() int64 { // s.mu.data might contain pointer tyeps, so we subtract its shallow size and // include the actual size. - dataSize := -int64(unsafe.Sizeof(roachpb.StatementStatistics{})) + + dataSize := -int64(unsafe.Sizeof(appstatspb.StatementStatistics{})) + int64(s.mu.data.Size()) return stmtStatsShallowSize + databaseNameSize + dataSize @@ -456,7 +456,7 @@ func (s *stmtStats) recordExecStats(stats execstats.QueryLevelStats) { s.mu.data.ExecStats.CPUSQLNanos.Record(count, float64(stats.CPUTime.Nanoseconds())) } -func (s *stmtStats) mergeStatsLocked(statistics *roachpb.CollectedStatementStatistics) { +func (s *stmtStats) mergeStatsLocked(statistics *appstatspb.CollectedStatementStatistics) { // This handles all the statistics fields. s.mu.data.Add(&statistics.Stats) @@ -486,12 +486,12 @@ func (s *Container) getStatsForStmt( database string, failed bool, planHash uint64, - transactionFingerprintID roachpb.TransactionFingerprintID, + transactionFingerprintID appstatspb.TransactionFingerprintID, createIfNonexistent bool, ) ( stats *stmtStats, key stmtKey, - stmtFingerprintID roachpb.StmtFingerprintID, + stmtFingerprintID appstatspb.StmtFingerprintID, created bool, throttled bool, ) { @@ -524,7 +524,7 @@ func (s *Container) getStatsForStmt( // If createIfNonexistent flag is set to true, then a new entry is created in // the Container if it does not yet exist. func (s *Container) getStatsForStmtWithKey( - key stmtKey, stmtFingerprintID roachpb.StmtFingerprintID, createIfNonexistent bool, + key stmtKey, stmtFingerprintID appstatspb.StmtFingerprintID, createIfNonexistent bool, ) (stats *stmtStats, created, throttled bool) { s.mu.Lock() defer s.mu.Unlock() @@ -532,7 +532,7 @@ func (s *Container) getStatsForStmtWithKey( } func (s *Container) getStatsForStmtWithKeyLocked( - key stmtKey, stmtFingerprintID roachpb.StmtFingerprintID, createIfNonexistent bool, + key stmtKey, stmtFingerprintID appstatspb.StmtFingerprintID, createIfNonexistent bool, ) (stats *stmtStats, created, throttled bool) { // Retrieve the per-statement statistic object, and create it if it // doesn't exist yet. @@ -564,8 +564,8 @@ func (s *Container) getStatsForStmtWithKeyLocked( } func (s *Container) getStatsForTxnWithKey( - key roachpb.TransactionFingerprintID, - stmtFingerprintIDs []roachpb.StmtFingerprintID, + key appstatspb.TransactionFingerprintID, + stmtFingerprintIDs []appstatspb.StmtFingerprintID, createIfNonexistent bool, ) (stats *txnStats, created, throttled bool) { s.mu.Lock() @@ -575,8 +575,8 @@ func (s *Container) getStatsForTxnWithKey( } func (s *Container) getStatsForTxnWithKeyLocked( - key roachpb.TransactionFingerprintID, - stmtFingerprintIDs []roachpb.StmtFingerprintID, + key appstatspb.TransactionFingerprintID, + stmtFingerprintIDs []appstatspb.StmtFingerprintID, createIfNonexistent bool, ) (stats *txnStats, created, throttled bool) { // Retrieve the per-transaction statistic object, and create it if it doesn't @@ -637,7 +637,7 @@ func (s *Container) Clear(ctx context.Context) { // Clear the map, to release the memory; make the new map somewhat already // large for the likely future workload. s.mu.stmts = make(map[stmtKey]*stmtStats, len(s.mu.stmts)/2) - s.mu.txns = make(map[roachpb.TransactionFingerprintID]*txnStats, len(s.mu.txns)/2) + s.mu.txns = make(map[appstatspb.TransactionFingerprintID]*txnStats, len(s.mu.txns)/2) s.mu.sampledPlanMetadataCache = make(map[sampledPlanKey]time.Time, len(s.mu.sampledPlanMetadataCache)/2) } @@ -662,12 +662,12 @@ func (s *Container) freeLocked(ctx context.Context) { func (s *Container) MergeApplicationStatementStats( ctx context.Context, other sqlstats.ApplicationStats, - transformer func(*roachpb.CollectedStatementStatistics), + transformer func(*appstatspb.CollectedStatementStatistics), ) (discardedStats uint64) { if err := other.IterateStatementStats( ctx, &sqlstats.IteratorOptions{}, - func(ctx context.Context, statistics *roachpb.CollectedStatementStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedStatementStatistics) error { if transformer != nil { transformer(statistics) } @@ -718,7 +718,7 @@ func (s *Container) MergeApplicationTransactionStats( if err := other.IterateTransactionStats( ctx, &sqlstats.IteratorOptions{}, - func(ctx context.Context, statistics *roachpb.CollectedTransactionStatistics) error { + func(ctx context.Context, statistics *appstatspb.CollectedTransactionStatistics) error { txnStats, _, throttled := s.getStatsForTxnWithKey( statistics.TransactionFingerprintID, @@ -817,10 +817,10 @@ func (s *Container) Add(ctx context.Context, other *Container) (err error) { } // Do what we did above for the statMap for the txn Map now. - txnMap := func() map[roachpb.TransactionFingerprintID]*txnStats { + txnMap := func() map[appstatspb.TransactionFingerprintID]*txnStats { other.mu.Lock() defer other.mu.Unlock() - txnMap := make(map[roachpb.TransactionFingerprintID]*txnStats) + txnMap := make(map[appstatspb.TransactionFingerprintID]*txnStats) for k, v := range other.mu.txns { txnMap[k] = v } @@ -884,14 +884,14 @@ func (s *Container) Add(ctx context.Context, other *Container) (err error) { } // Create a copy of the other's transactions statistics. - txnStats := func() roachpb.TxnStats { + txnStats := func() appstatspb.TxnStats { other.txnCounts.mu.Lock() defer other.txnCounts.mu.Unlock() return other.txnCounts.mu.TxnStats }() // Merge the transaction stats. - func(txnStats roachpb.TxnStats) { + func(txnStats appstatspb.TxnStats) { s.txnCounts.mu.Lock() defer s.txnCounts.mu.Unlock() s.txnCounts.mu.TxnStats.Add(txnStats) @@ -954,12 +954,12 @@ type transactionCounts struct { mu struct { syncutil.Mutex // TODO(arul): Can we rename this without breaking stuff? - roachpb.TxnStats + appstatspb.TxnStats } } -func constructStatementFingerprintIDFromStmtKey(key stmtKey) roachpb.StmtFingerprintID { - return roachpb.ConstructStatementFingerprintID( +func constructStatementFingerprintIDFromStmtKey(key stmtKey) appstatspb.StmtFingerprintID { + return appstatspb.ConstructStatementFingerprintID( key.stmtNoConstants, key.failed, key.implicitTxn, key.database, ) } diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go index 7369618fca4d..603c65b479e9 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go @@ -16,6 +16,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/execstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights" @@ -34,7 +35,7 @@ var ( // ErrExecStatsFingerprintFlushed is returned from the Container when the // stats object for the fingerprint has been flushed to system table before - // the roachpb.ExecStats can be recorded. + // the appstatspb.ExecStats can be recorded. ErrExecStatsFingerprintFlushed = errors.New("stmtStats flushed before execution stats can be recorded") ) @@ -66,8 +67,8 @@ func getStatus(statementError error) insights.Statement_Status { // statistics into in-memory structs. It is unrelated to the stmtErr in the // arguments. func (s *Container) RecordStatement( - ctx context.Context, key roachpb.StatementStatisticsKey, value sqlstats.RecordedStmtStats, -) (roachpb.StmtFingerprintID, error) { + ctx context.Context, key appstatspb.StatementStatisticsKey, value sqlstats.RecordedStmtStats, +) (appstatspb.StmtFingerprintID, error) { createIfNonExistent := true // If the statement is below the latency threshold, or stats aren't being // recorded we don't need to create an entry in the stmts map for it. We do @@ -214,7 +215,7 @@ func (s *Container) RecordStatement( // RecordStatementExecStats implements sqlstats.Writer interface. func (s *Container) RecordStatementExecStats( - key roachpb.StatementStatisticsKey, stats execstats.QueryLevelStats, + key appstatspb.StatementStatisticsKey, stats execstats.QueryLevelStats, ) error { stmtStats, _, _, _, _ := s.getStatsForStmt( @@ -249,7 +250,7 @@ func (s *Container) ShouldSample( // RecordTransaction implements sqlstats.Writer interface and saves // per-transaction statistics. func (s *Container) RecordTransaction( - ctx context.Context, key roachpb.TransactionFingerprintID, value sqlstats.RecordedTxnStats, + ctx context.Context, key appstatspb.TransactionFingerprintID, value sqlstats.RecordedTxnStats, ) error { s.recordTransactionHighLevelStats(value.TransactionTimeSec, value.Committed, value.ImplicitTxn) diff --git a/pkg/sql/sqlstats/ssmemstorage/utils.go b/pkg/sql/sqlstats/ssmemstorage/utils.go index 26a1094a2027..fa1efc87017b 100644 --- a/pkg/sql/sqlstats/ssmemstorage/utils.go +++ b/pkg/sql/sqlstats/ssmemstorage/utils.go @@ -13,7 +13,7 @@ package ssmemstorage import ( "strings" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" ) type stmtList []stmtKey @@ -36,7 +36,7 @@ func (s stmtList) Less(i, j int) bool { return s[i].transactionFingerprintID < s[j].transactionFingerprintID } -type txnList []roachpb.TransactionFingerprintID +type txnList []appstatspb.TransactionFingerprintID func (t txnList) Len() int { return len(t) diff --git a/pkg/sql/sqlstats/ssprovider.go b/pkg/sql/sqlstats/ssprovider.go index bc123ded975c..c20c3bd7fd93 100644 --- a/pkg/sql/sqlstats/ssprovider.go +++ b/pkg/sql/sqlstats/ssprovider.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/execstats" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -31,11 +32,11 @@ import ( // transaction stats. type Writer interface { // RecordStatement records statistics for a statement. - RecordStatement(ctx context.Context, key roachpb.StatementStatisticsKey, value RecordedStmtStats) (roachpb.StmtFingerprintID, error) + RecordStatement(ctx context.Context, key appstatspb.StatementStatisticsKey, value RecordedStmtStats) (appstatspb.StmtFingerprintID, error) // RecordStatementExecStats records execution statistics for a statement. // This is sampled and not recorded for every single statement. - RecordStatementExecStats(key roachpb.StatementStatisticsKey, stats execstats.QueryLevelStats) error + RecordStatementExecStats(key appstatspb.StatementStatisticsKey, stats execstats.QueryLevelStats) error // ShouldSample returns two booleans, the first one indicates whether we // ever sampled (i.e. collected statistics for) the given combination of @@ -44,7 +45,7 @@ type Writer interface { ShouldSample(fingerprint string, implicitTxn bool, database string) (previouslySampled, savePlanForStats bool) // RecordTransaction records statistics for a transaction. - RecordTransaction(ctx context.Context, key roachpb.TransactionFingerprintID, value RecordedTxnStats) error + RecordTransaction(ctx context.Context, key appstatspb.TransactionFingerprintID, value RecordedTxnStats) error } // Reader provides methods to retrieve transaction/statement statistics from @@ -82,7 +83,7 @@ type ApplicationStats interface { MergeApplicationStatementStats( ctx context.Context, other ApplicationStats, - transformer func(statistics *roachpb.CollectedStatementStatistics), + transformer func(statistics *appstatspb.CollectedStatementStatistics), ) uint64 // MergeApplicationTransactionStats merges the other application's transaction @@ -121,18 +122,18 @@ type IteratorOptions struct { // StatementVisitor is the callback that is invoked when caller iterate through // all statement statistics using IterateStatementStats(). If an error is // encountered when calling the visitor, the iteration is aborted. -type StatementVisitor func(context.Context, *roachpb.CollectedStatementStatistics) error +type StatementVisitor func(context.Context, *appstatspb.CollectedStatementStatistics) error // TransactionVisitor is the callback that is invoked when caller iterate through // all transaction statistics using IterateTransactionStats(). If an error is // encountered when calling the visitor, the iteration is aborted. -type TransactionVisitor func(context.Context, *roachpb.CollectedTransactionStatistics) error +type TransactionVisitor func(context.Context, *appstatspb.CollectedTransactionStatistics) error // AggregatedTransactionVisitor is the callback invoked when iterate through // transaction statistics collected at the application level using // IterateAggregatedTransactionStats(). If an error is encountered when calling // the visitor, the iteration is aborted. -type AggregatedTransactionVisitor func(appName string, statistics *roachpb.TxnStats) error +type AggregatedTransactionVisitor func(appName string, statistics *appstatspb.TxnStats) error // StatsCollector is an interface that collects statistics for transactions and // statements for the entire lifetime of a session. @@ -159,7 +160,7 @@ type StatsCollector interface { // fingerprint ID is now available. StatsCollector will now go back to update // the transaction fingerprint ID field of all the statement statistics for that // txn. - EndTransaction(ctx context.Context, transactionFingerprintID roachpb.TransactionFingerprintID) + EndTransaction(ctx context.Context, transactionFingerprintID appstatspb.TransactionFingerprintID) // UpgradeImplicitTxn informs the StatsCollector that the current txn has been // upgraded to an explicit transaction, thus all previously recorded statements @@ -209,7 +210,7 @@ type RecordedStmtStats struct { RowsWritten int64 Nodes []int64 StatementType tree.StatementType - Plan *roachpb.ExplainTreePlanNode + Plan *appstatspb.ExplainTreePlanNode PlanGist string StatementError error IndexRecommendations []string @@ -233,7 +234,7 @@ type RecordedTxnStats struct { ImplicitTxn bool RetryCount int64 AutoRetryReason error - StatementFingerprintIDs []roachpb.StmtFingerprintID + StatementFingerprintIDs []appstatspb.StmtFingerprintID ServiceLatency time.Duration RetryLatency time.Duration CommitLatency time.Duration diff --git a/pkg/sql/sqltestutils/BUILD.bazel b/pkg/sql/sqltestutils/BUILD.bazel index b03f2f1b167a..29463e31ffc8 100644 --- a/pkg/sql/sqltestutils/BUILD.bazel +++ b/pkg/sql/sqltestutils/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/server", "//pkg/server/diagnostics", "//pkg/sql", + "//pkg/sql/appstatspb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/desctestutils", "//pkg/sql/parser", diff --git a/pkg/sql/sqltestutils/telemetry.go b/pkg/sql/sqltestutils/telemetry.go index df75e7db47a3..d9a4d957be3b 100644 --- a/pkg/sql/sqltestutils/telemetry.go +++ b/pkg/sql/sqltestutils/telemetry.go @@ -22,10 +22,10 @@ import ( "text/tabwriter" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/server/diagnostics" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -366,8 +366,8 @@ func formatTableDescriptor(desc *descpb.TableDescriptor) string { return tp.String() } -func formatSQLStats(stats []roachpb.CollectedStatementStatistics) string { - bucketByApp := make(map[string][]roachpb.CollectedStatementStatistics) +func formatSQLStats(stats []appstatspb.CollectedStatementStatistics) string { + bucketByApp := make(map[string][]appstatspb.CollectedStatementStatistics) for i := range stats { s := &stats[i] diff --git a/pkg/sql/telemetry_logging_test.go b/pkg/sql/telemetry_logging_test.go index 661204ba4941..9fe961d274b9 100644 --- a/pkg/sql/telemetry_logging_test.go +++ b/pkg/sql/telemetry_logging_test.go @@ -22,7 +22,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/execstats" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -452,7 +452,7 @@ func TestTelemetryLogging(t *testing.T) { if !strings.Contains(e.Message, "\"Database\":\""+databaseName+"\"") { t.Errorf("expected to find Database: %s", databaseName) } - stmtFingerprintID := roachpb.ConstructStatementFingerprintID(tc.queryNoConstants, tc.expectedErr != "", true, databaseName) + stmtFingerprintID := appstatspb.ConstructStatementFingerprintID(tc.queryNoConstants, tc.expectedErr != "", true, databaseName) if !strings.Contains(e.Message, "\"StatementFingerprintID\":"+strconv.FormatUint(uint64(stmtFingerprintID), 10)) { t.Errorf("expected to find StatementFingerprintID: %v", stmtFingerprintID) } diff --git a/pkg/sql/txn_fingerprint_id_cache.go b/pkg/sql/txn_fingerprint_id_cache.go index 7f793e4e46db..a2d0ab821d31 100644 --- a/pkg/sql/txn_fingerprint_id_cache.go +++ b/pkg/sql/txn_fingerprint_id_cache.go @@ -13,9 +13,9 @@ package sql import ( "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/util/cache" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -76,7 +76,7 @@ func NewTxnFingerprintIDCache( // Add adds a TxnFingerprintID to the cache, truncating the cache to the cache's capacity // if necessary. -func (b *TxnFingerprintIDCache) Add(value roachpb.TransactionFingerprintID) error { +func (b *TxnFingerprintIDCache) Add(value appstatspb.TransactionFingerprintID) error { b.mu.Lock() defer b.mu.Unlock() @@ -91,7 +91,7 @@ func (b *TxnFingerprintIDCache) Add(value roachpb.TransactionFingerprintID) erro // GetAllTxnFingerprintIDs returns a slice of all TxnFingerprintIDs in the cache. // The cache may be truncated if the capacity was updated to a smaller size. -func (b *TxnFingerprintIDCache) GetAllTxnFingerprintIDs() []roachpb.TransactionFingerprintID { +func (b *TxnFingerprintIDCache) GetAllTxnFingerprintIDs() []appstatspb.TransactionFingerprintID { b.mu.Lock() defer b.mu.Unlock() @@ -101,11 +101,11 @@ func (b *TxnFingerprintIDCache) GetAllTxnFingerprintIDs() []roachpb.TransactionF size = capacity } - txnFingerprintIDs := make([]roachpb.TransactionFingerprintID, 0, size) - txnFingerprintIDsRemoved := make([]roachpb.TransactionFingerprintID, 0) + txnFingerprintIDs := make([]appstatspb.TransactionFingerprintID, 0, size) + txnFingerprintIDsRemoved := make([]appstatspb.TransactionFingerprintID, 0) b.mu.cache.Do(func(entry *cache.Entry) { - id := entry.Value.(roachpb.TransactionFingerprintID) + id := entry.Value.(appstatspb.TransactionFingerprintID) if int64(len(txnFingerprintIDs)) == size { txnFingerprintIDsRemoved = append(txnFingerprintIDsRemoved, id) diff --git a/pkg/sql/txn_fingerprint_id_cache_test.go b/pkg/sql/txn_fingerprint_id_cache_test.go index a21dde3bb3f2..7ec21e74c87c 100644 --- a/pkg/sql/txn_fingerprint_id_cache_test.go +++ b/pkg/sql/txn_fingerprint_id_cache_test.go @@ -18,9 +18,9 @@ import ( "strconv" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" @@ -75,7 +75,7 @@ func TestTxnFingerprintIDCacheDataDriven(t *testing.T) { id, err := strconv.ParseUint(idStr, 10, 64) require.NoError(t, err) - txnFingerprintID := roachpb.TransactionFingerprintID(id) + txnFingerprintID := appstatspb.TransactionFingerprintID(id) err = txnFingerprintIDCache.Add(txnFingerprintID) require.NoError(t, err) @@ -105,7 +105,7 @@ func TestTxnFingerprintIDCache(t *testing.T) { ctx := context.Background() - txnFingerprintIDsRecorded := make([]roachpb.TransactionFingerprintID, 0) + txnFingerprintIDsRecorded := make([]appstatspb.TransactionFingerprintID, 0) appName := "testTxnFingerprintIDCache" params, _ := tests.CreateTestServerParams() @@ -113,7 +113,7 @@ func TestTxnFingerprintIDCache(t *testing.T) { BeforeTxnStatsRecorded: func( sessionData *sessiondata.SessionData, _ uuid.UUID, - txnFingerprintID roachpb.TransactionFingerprintID, + txnFingerprintID appstatspb.TransactionFingerprintID, ) { if !sessionData.Internal { // Record every query we issue through our sql connection. diff --git a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts index 5d6b908371f9..0dc48ebde9af 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts @@ -24,7 +24,7 @@ import IExplainTreePlanNode = protos.cockroach.sql.IExplainTreePlanNode; import ISensitiveInfo = protos.cockroach.sql.ISensitiveInfo; // record is implemented here so we can write the below test as a direct -// analog of the one in pkg/roachpb/app_stats_test.go. It's here rather +// analog of the one in pkg/sql/appstatspb/app_stats_test.go. It's here rather // than in the main source file because we don't actually need it for the // application to use. function record(l: NumericStat, count: number, val: number) {