Skip to content

Commit

Permalink
sql: refactor SQL stats collection to its own package
Browse files Browse the repository at this point in the history
Previously, SQL stats collection is implemented within the sql
package. However, this is not ideal because it introduces a
high-level coupling between the SQLServer and rest of subsystems
within CRDB. As we move to persisted SQL stats, the SQL stats
collection will experience an increase of complexity.

This commit moves SQL stats collection into its own subsystem and
introduced intra-subsystem interfaces to decouple the various
components within the subsystem.

Release note: None

Closes #65699
  • Loading branch information
Azhng committed Jun 9, 2021
1 parent 8be5c15 commit 06f6874
Show file tree
Hide file tree
Showing 30 changed files with 2,353 additions and 1,670 deletions.
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -319,6 +319,7 @@ go_test(
"//pkg/sql/catalog/descpb",
"//pkg/sql/execinfrapb",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlstats",
"//pkg/sql/tests",
"//pkg/sqlmigrations",
"//pkg/storage",
Expand Down
8 changes: 7 additions & 1 deletion pkg/server/diagnostics/reporter.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,7 +251,13 @@ func (r *Reporter) CreateReport(
}
}

info.SqlStats = r.SQLServer.GetScrubbedReportingStats()
info.SqlStats, err = r.SQLServer.GetScrubbedReportingStats(ctx)
if err != nil {
if log.V(2 /* level */) {
log.Warningf(ctx, "unexpected error encountered when getting scrubbed reporting stats: %s", err)
}
}

return &info
}

Expand Down
16 changes: 12 additions & 4 deletions pkg/server/statements.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ func (s *statusServer) Statements(
return nil, status.Errorf(codes.InvalidArgument, err.Error())
}
if local {
return statementsLocal(s.gossip.NodeID, s.admin.server.sqlServer)
return statementsLocal(ctx, s.gossip.NodeID, s.admin.server.sqlServer)
}
status, err := s.dialNode(ctx, requestedNodeID)
if err != nil {
Expand Down Expand Up @@ -89,10 +89,18 @@ func (s *statusServer) Statements(
}

func statementsLocal(
nodeID *base.NodeIDContainer, sqlServer *SQLServer,
ctx context.Context, nodeID *base.NodeIDContainer, sqlServer *SQLServer,
) (*serverpb.StatementsResponse, error) {
stmtStats := sqlServer.pgServer.SQLServer.GetUnscrubbedStmtStats()
txnStats := sqlServer.pgServer.SQLServer.GetUnscrubbedTxnStats()
stmtStats, err := sqlServer.pgServer.SQLServer.GetUnscrubbedStmtStats(ctx)
if err != nil {
return nil, err
}

txnStats, err := sqlServer.pgServer.SQLServer.GetUnscrubbedTxnStats(ctx)
if err != nil {
return nil, err
}

lastReset := sqlServer.pgServer.SQLServer.GetStmtStatsLastReset()

resp := &serverpb.StatementsResponse{
Expand Down
28 changes: 20 additions & 8 deletions pkg/server/stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"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/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -74,7 +74,9 @@ CREATE TABLE t.test (x INT PRIMARY KEY);
s.DiagnosticsReporter().(*diagnostics.Reporter).ReportDiagnostics(ctx)

// Ensure that our SQL statement data was not affected by the telemetry report.
stats := sqlServer.GetScrubbedStmtStats()
stats, err := sqlServer.GetScrubbedStmtStats(ctx)
require.NoError(t, err)

foundStat := false
for _, stat := range stats {
if stat.Key.Query == "INSERT INTO _ VALUES ($1)" {
Expand All @@ -98,7 +100,7 @@ func TestEnsureSQLStatsAreFlushedForTelemetry(t *testing.T) {
params.Settings = cluster.MakeClusterSettings()
// Set the SQL stat refresh rate very low so that SQL stats are continuously
// flushed into the telemetry reporting stats pool.
sql.SQLStatReset.Override(ctx, &params.Settings.SV, 10*time.Millisecond)
sqlstats.SQLStatReset.Override(ctx, &params.Settings.SV, 10*time.Millisecond)
s, sqlDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)

Expand Down Expand Up @@ -135,7 +137,9 @@ INSERT INTO t.test VALUES (2);

// We should also not find the stat in the SQL stats pool, since the SQL
// stats are getting flushed.
stats := sqlServer.GetScrubbedStmtStats()
stats, err := sqlServer.GetScrubbedStmtStats(ctx)
require.NoError(t, err)

for _, stat := range stats {
// These stats are scrubbed, so look for our scrubbed statement.
if strings.HasPrefix(stat.Key.Query, "INSERT INTO _ VALUES (_)") {
Expand Down Expand Up @@ -172,7 +176,9 @@ func TestSQLStatCollection(t *testing.T) {
}

// Collect stats from the SQL server and ensure our queries are present.
stats := sqlServer.GetScrubbedStmtStats()
stats, err := sqlServer.GetScrubbedStmtStats(ctx)
require.NoError(t, err)

foundStat := false
var sqlStatData roachpb.StatementStatistics

Expand All @@ -193,7 +199,9 @@ func TestSQLStatCollection(t *testing.T) {
sqlServer.ResetSQLStats(ctx)

// Query the reported statistics.
stats = sqlServer.GetScrubbedReportingStats()
stats, err = sqlServer.GetScrubbedReportingStats(ctx)
require.NoError(t, err)

foundStat = false
for _, stat := range stats {
if stat.Key.Query == "INSERT INTO _ VALUES (_)" {
Expand All @@ -218,7 +226,9 @@ func TestSQLStatCollection(t *testing.T) {
}

// Find and record the stats for our second query.
stats = sqlServer.GetScrubbedStmtStats()
stats, err = sqlServer.GetScrubbedStmtStats(ctx)
require.NoError(t, err)

foundStat = false
for _, stat := range stats {
if stat.Key.Query == "INSERT INTO _ VALUES (_)" {
Expand All @@ -235,7 +245,9 @@ func TestSQLStatCollection(t *testing.T) {
sqlServer.ResetSQLStats(ctx)

// Find our statement stat from the reported stats pool.
stats = sqlServer.GetScrubbedReportingStats()
stats, err = sqlServer.GetScrubbedReportingStats(ctx)
require.NoError(t, err)

foundStat = false
for _, stat := range stats {
if stat.Key.Query == "INSERT INTO _ VALUES (_)" {
Expand Down
5 changes: 3 additions & 2 deletions pkg/server/status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -1501,7 +1502,7 @@ func TestStatusAPITransactions(t *testing.T) {
t.Fatal(err)
}

// Construct a map of all the statement IDs.
// Construct a map of all the statement fingerprint IDs.
statementFingerprintIDs := make(map[roachpb.StmtFingerprintID]bool, len(resp.Statements))
for _, respStatement := range resp.Statements {
statementFingerprintIDs[respStatement.ID] = true
Expand Down Expand Up @@ -1569,7 +1570,7 @@ func TestStatusAPITransactionStatementFingerprintIDsTruncation(t *testing.T) {
thirdServerSQL.Exec(t, `CREATE DATABASE db; CREATE TABLE db.t();`)
thirdServerSQL.Exec(t, fmt.Sprintf(`SET application_name = "%s"`, testingApp))

maxStmtFingerprintIDsLen := int(sql.TxnStatsNumStmtFingerprintIDsToRecord.Get(
maxStmtFingerprintIDsLen := int(sqlstats.TxnStatsNumStmtFingerprintIDsToRecord.Get(
&firstServerProto.ExecutorConfig().(sql.ExecutorConfig).Settings.SV))

// Construct 2 transaction queries that include an absurd number of statements.
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/tenant_status.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ func (t *tenantStatusServer) Statements(
}
// Use a dummy value here until pod-to-pod communication is implemented since tenant status server
// does not have concept of node.
resp, err := statementsLocal(&base.NodeIDContainer{}, t.sqlServer)
resp, err := statementsLocal(ctx, &base.NodeIDContainer{}, t.sqlServer)

if err != nil {
return nil, err
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ go_library(
"alter_table_set_schema.go",
"alter_type.go",
"analyze_expr.go",
"app_stats.go",
"apply_join.go",
"authorization.go",
"backfill.go",
Expand Down Expand Up @@ -346,6 +345,8 @@ go_library(
"//pkg/sql/span",
"//pkg/sql/sqlerrors",
"//pkg/sql/sqlliveness",
"//pkg/sql/sqlstats",
"//pkg/sql/sqlstats/sslocal",
"//pkg/sql/sqltelemetry",
"//pkg/sql/sqlutil",
"//pkg/sql/stats",
Expand Down Expand Up @@ -572,6 +573,7 @@ go_test(
"//pkg/sql/sessiondatapb",
"//pkg/sql/sessionphase",
"//pkg/sql/span",
"//pkg/sql/sqlstats",
"//pkg/sql/sqltestutils",
"//pkg/sql/sqlutil",
"//pkg/sql/stats",
Expand Down
Loading

0 comments on commit 06f6874

Please sign in to comment.