diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index f9dc20d6824b..274ad8b90781 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -448,6 +448,9 @@ func makeServerMetrics(cfg *ExecutorConfig) ServerMetrics { MetaSQLStatsFlushDuration, 6*metricsSampleInterval, ), SQLStatsRemovedRows: metric.NewCounter(MetaSQLStatsRemovedRows), + SQLTxnStatsCollectionOverhead: metric.NewLatency( + MetaSQLTxnStatsCollectionOverhead, 6*metricsSampleInterval, + ), }, } } diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 02bf2441bcca..ea4297c5fc8b 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -1984,6 +1984,14 @@ func (ex *connExecutor) recordTransaction( // meaningful. return nil } + recordingStart := timeutil.Now() + defer func() { + recordingOverhead := timeutil.Since(recordingStart) + ex.server. + ServerMetrics. + StatsMetrics. + SQLTxnStatsCollectionOverhead.RecordValue(recordingOverhead.Nanoseconds()) + }() txnEnd := timeutil.Now() txnTime := txnEnd.Sub(txnStart) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index b19c36936001..5121d85d271a 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1043,6 +1043,12 @@ var ( Measurement: "SQL Stats Cleanup", Unit: metric.Unit_COUNT, } + MetaSQLTxnStatsCollectionOverhead = metric.Metadata{ + Name: "sql.stats.txn_stats_collection.duration", + Help: "Time took in nanoseconds to collect transaction stats", + Measurement: "SQL Transaction Stats Collection Overhead", + Unit: metric.Unit_NANOSECONDS, + } MetaTxnRowsWrittenLog = metric.Metadata{ Name: "sql.guardrails.transaction_rows_written_log.count", Help: "Number of transactions logged because of transaction_rows_written_log guardrail", diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 06e63380d045..4f4ee3ab6132 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -76,6 +76,8 @@ type StatsMetrics struct { SQLStatsFlushFailure *metric.Counter SQLStatsFlushDuration *metric.Histogram SQLStatsRemovedRows *metric.Counter + + SQLTxnStatsCollectionOverhead *metric.Histogram } // StatsMetrics is part of the metric.Struct interface. diff --git a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel index 517b906f3408..d5eabe909dfd 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel +++ b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel @@ -53,6 +53,7 @@ go_library( go_test( name = "persistedsqlstats_test", srcs = [ + "bench_test.go", "compaction_test.go", "controller_test.go", "datadriven_test.go", @@ -86,6 +87,7 @@ go_test( "//pkg/sql/tests", "//pkg/testutils", "//pkg/testutils/serverutils", + "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/encoding", diff --git a/pkg/sql/sqlstats/persistedsqlstats/bench_test.go b/pkg/sql/sqlstats/persistedsqlstats/bench_test.go new file mode 100644 index 000000000000..ebb10adde996 --- /dev/null +++ b/pkg/sql/sqlstats/persistedsqlstats/bench_test.go @@ -0,0 +1,81 @@ +// Copyright 2022 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 persistedsqlstats_test + +import ( + "context" + gosql "database/sql" + "fmt" + "sync" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" +) + +func BenchmarkConcurrentSelect1(b *testing.B) { + skip.UnderShort(b) + defer log.Scope(b).Close(b) + ctx := context.Background() + + for _, numOfConcurrentConn := range []int{24, 48, 64} { + b.Run(fmt.Sprintf("concurrentConn=%d", numOfConcurrentConn), func(b *testing.B) { + s, db, _ := serverutils.StartServer(b, base.TestServerArgs{}) + sqlServer := s.SQLServer().(*sql.Server) + defer s.Stopper().Stop(ctx) + + starter := make(chan struct{}) + latencyChan := make(chan float64, numOfConcurrentConn) + defer close(latencyChan) + + var wg sync.WaitGroup + for connIdx := 0; connIdx < numOfConcurrentConn; connIdx++ { + sqlConn, err := db.Conn(ctx) + if err != nil { + b.Fatalf("unexpected error creating db conn: %s", err) + } + wg.Add(1) + + go func(conn *gosql.Conn, idx int) { + defer wg.Done() + runner := sqlutils.MakeSQLRunner(conn) + <-starter + + start := timeutil.Now() + for i := 0; i < b.N; i++ { + runner.Exec(b, "SELECT 1") + } + duration := timeutil.Since(start) + latencyChan <- float64(duration.Milliseconds()) / float64(b.N) + }(sqlConn, connIdx) + } + + close(starter) + wg.Wait() + + var totalLat float64 + for i := 0; i < numOfConcurrentConn; i++ { + totalLat += <-latencyChan + } + b.ReportMetric( + sqlServer.ServerMetrics. + StatsMetrics. + SQLTxnStatsCollectionOverhead. + Snapshot().Mean(), + "overhead(ns/op)") + }) + } +} diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 8396055e53b5..942bcbda7e42 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -1862,6 +1862,10 @@ var charts = []sectionDescription{ Title: "Number of stale statement/transaction roles removed by cleanup job", Metrics: []string{"sql.stats.cleanup.rows_removed"}, }, + { + Title: "Time took in nanoseconds to collect transaction stats", + Metrics: []string{"sql.stats.txn_stats_collection.duration"}, + }, }, }, { diff --git a/pkg/ts/catalog/metrics.go b/pkg/ts/catalog/metrics.go index 03b93925cc35..301f662c4608 100644 --- a/pkg/ts/catalog/metrics.go +++ b/pkg/ts/catalog/metrics.go @@ -90,6 +90,7 @@ var histogramMetricsNames = map[string]struct{}{ "txnwaitqueue.pusher.wait_time": {}, "txnwaitqueue.query.wait_time": {}, "raft.process.applycommitted.latency": {}, + "sql.stats.txn_stats_collection.duration": {}, } func allInternalTSMetricsNames() []string {