diff --git a/pkg/sql/sqlstats/sslocal/BUILD.bazel b/pkg/sql/sqlstats/sslocal/BUILD.bazel index d18b27795d2f..8f30f0217da6 100644 --- a/pkg/sql/sqlstats/sslocal/BUILD.bazel +++ b/pkg/sql/sqlstats/sslocal/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "sql_stats.go", "sql_stats_controller.go", + "sslocal_iterator.go", "sslocal_provider.go", "sslocal_sink.go", "sslocal_stats_collector.go", diff --git a/pkg/sql/sqlstats/sslocal/sql_stats.go b/pkg/sql/sqlstats/sslocal/sql_stats.go index 57b1a174d68d..0b99ee85a323 100644 --- a/pkg/sql/sqlstats/sslocal/sql_stats.go +++ b/pkg/sql/sqlstats/sslocal/sql_stats.go @@ -129,6 +129,7 @@ func (s *SQLStats) getStatsForApplication(appName string) *ssmemstorage.Containe &s.atomic.uniqueStmtFingerprintCount, &s.atomic.uniqueTxnFingerprintCount, s.mu.mon, + appName, ) s.mu.apps[appName] = a return a diff --git a/pkg/sql/sqlstats/sslocal/sslocal_iterator.go b/pkg/sql/sqlstats/sslocal/sslocal_iterator.go new file mode 100644 index 000000000000..883dff936a92 --- /dev/null +++ b/pkg/sql/sqlstats/sslocal/sslocal_iterator.go @@ -0,0 +1,121 @@ +// Copyright 2021 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 sslocal + +import ( + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" + "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/ssmemstorage" +) + +type baseIterator struct { + sqlStats *SQLStats + idx int + appNames []string + options *sqlstats.IteratorOptions +} + +// StmtStatsIterator is an iterator that can be used to iterate over the +// statement statistics stored in SQLStats. +type StmtStatsIterator struct { + baseIterator + curIter *ssmemstorage.StmtStatsIterator +} + +// NewStmtStatsIterator returns a new instance of the StmtStatsIterator. +func NewStmtStatsIterator(s *SQLStats, options *sqlstats.IteratorOptions) *StmtStatsIterator { + appNames := s.getAppNames(options.SortedAppNames) + + return &StmtStatsIterator{ + baseIterator: baseIterator{ + sqlStats: s, + idx: -1, + appNames: appNames, + options: options, + }, + } +} + +// Next increments the internal counter of the StmtStatsIterator. It returns +// true if the following Cur() call will be valid, false otherwise. +func (s *StmtStatsIterator) Next() bool { + // If we haven't called Next() for the first time or our current child + // iterator has finished iterator, then we increment s.idx. + if s.curIter == nil || !s.curIter.Next() { + s.idx++ + if s.idx >= len(s.appNames) { + return false + } + statsContainer := s.sqlStats.getStatsForApplication(s.appNames[s.idx]) + s.curIter = statsContainer.StmtStatsIterator(s.options) + return s.Next() + } + + // This happens when our child iterator is valid and s.curIter.Next() call + // is true. + return true +} + +// Cur returns the roachpb.CollectedStatementStatistics at the current internal +// counter. +func (s *StmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { + return s.curIter.Cur() +} + +// TxnStatsIterator is an iterator that can be used to iterate over the +// statement statistics stored in SQLStats. +type TxnStatsIterator struct { + baseIterator + curIter *ssmemstorage.TxnStatsIterator +} + +// NewTxnStatsIterator returns a new instance of the TxnStatsIterator. +func NewTxnStatsIterator(s *SQLStats, options *sqlstats.IteratorOptions) *TxnStatsIterator { + appNames := s.getAppNames(options.SortedAppNames) + + return &TxnStatsIterator{ + baseIterator: baseIterator{ + sqlStats: s, + idx: -1, + appNames: appNames, + options: options, + }, + } +} + +// Next increments the internal counter of the TxnStatsIterator. It returns +// true if the following Cur() call will be valid, false otherwise. +func (t *TxnStatsIterator) Next() bool { + // If we haven't called Next() for the first time or our current child + // iterator has finished iterator, then we increment s.idx. + if t.curIter == nil || !t.curIter.Next() { + t.idx++ + if t.idx >= len(t.appNames) { + return false + } + statsContainer := t.sqlStats.getStatsForApplication(t.appNames[t.idx]) + t.curIter = statsContainer.TxnStatsIterator(t.options) + return t.Next() + } + + // This happens when our child iterator is valid and s.curIter.Next() call + // is true. + return true +} + +// Cur returns the roachpb.CollectedTransactionStatistics at the current internal +// counter. +func (t *TxnStatsIterator) Cur() ( + roachpb.TransactionFingerprintID, + *roachpb.CollectedTransactionStatistics, +) { + return t.curIter.Cur() +} diff --git a/pkg/sql/sqlstats/sslocal/sslocal_provider.go b/pkg/sql/sqlstats/sslocal/sslocal_provider.go index 10d1affd533e..bca0246628fa 100644 --- a/pkg/sql/sqlstats/sslocal/sslocal_provider.go +++ b/pkg/sql/sqlstats/sslocal/sslocal_provider.go @@ -111,6 +111,7 @@ func (s *SQLStats) GetWriterForApplication(appName string) sqlstats.Writer { &s.atomic.uniqueStmtFingerprintCount, &s.atomic.uniqueTxnFingerprintCount, s.mu.mon, + appName, ) s.mu.apps[appName] = a return a @@ -127,36 +128,45 @@ func (s *SQLStats) GetLastReset() time.Time { func (s *SQLStats) IterateStatementStats( ctx context.Context, options *sqlstats.IteratorOptions, visitor sqlstats.StatementVisitor, ) error { - appNames := s.getAppNames(options.SortedAppNames) + iter := s.StmtStatsIterator(options) - for _, appName := range appNames { - statsContainer := s.getStatsForApplication(appName) - - err := statsContainer.IterateStatementStats(ctx, appName, options.SortedKey, visitor) - if err != nil { - return fmt.Errorf("sql stats iteration abort: %s", err) + for iter.Next() { + if err := visitor(ctx, iter.Cur()); err != nil { + return err } } + return nil } +// StmtStatsIterator returns an instance of sslocal.StmtStatsIterator for +// the current SQLStats. +func (s *SQLStats) StmtStatsIterator(options *sqlstats.IteratorOptions) *StmtStatsIterator { + return NewStmtStatsIterator(s, options) +} + // IterateTransactionStats implements sqlstats.Provider interface. func (s *SQLStats) IterateTransactionStats( ctx context.Context, options *sqlstats.IteratorOptions, visitor sqlstats.TransactionVisitor, ) error { - appNames := s.getAppNames(options.SortedAppNames) + iter := s.TxnStatsIterator(options) - for _, appName := range appNames { - statsContainer := s.getStatsForApplication(appName) - - err := statsContainer.IterateTransactionStats(ctx, appName, options.SortedKey, visitor) - if err != nil { - return fmt.Errorf("sql stats iteration abort: %s", err) + for iter.Next() { + fingerprintID, stats := iter.Cur() + if err := visitor(ctx, fingerprintID, stats); err != nil { + return err } } + return nil } +// TxnStatsIterator returns an instance of sslocal.TxnStatsIterator for +// the current SQLStats. +func (s *SQLStats) TxnStatsIterator(options *sqlstats.IteratorOptions) *TxnStatsIterator { + return NewTxnStatsIterator(s, options) +} + // IterateAggregatedTransactionStats implements sqlstats.Provider interface. func (s *SQLStats) IterateAggregatedTransactionStats( _ context.Context, diff --git a/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel b/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel index 4c2eac73bbbe..5cec5b5f09a9 100644 --- a/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel +++ b/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "ssmemstorage", srcs = [ + "ss_mem_iterator.go", "ss_mem_storage.go", "ss_mem_writer.go", "utils.go", diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go new file mode 100644 index 000000000000..1af49cad2753 --- /dev/null +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go @@ -0,0 +1,169 @@ +// Copyright 2021 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 ssmemstorage + +import ( + "sort" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" +) + +type baseIterator struct { + container *Container + idx int +} + +// StmtStatsIterator is an iterator that iterates over the statement statistics +// inside of a ssmemstorage.Container. +type StmtStatsIterator struct { + baseIterator + stmtKeys stmtList +} + +// NewStmtStatsIterator returns a StmtStatsIterator. +func NewStmtStatsIterator( + container *Container, options *sqlstats.IteratorOptions, +) *StmtStatsIterator { + var stmtKeys stmtList + container.mu.Lock() + for k := range container.mu.stmts { + stmtKeys = append(stmtKeys, k) + } + container.mu.Unlock() + if options.SortedKey { + sort.Sort(stmtKeys) + } + + return &StmtStatsIterator{ + baseIterator: baseIterator{ + container: container, + idx: -1, + }, + stmtKeys: stmtKeys, + } +} + +// Next increments the internal counter of the StmtStatsIterator. It returns +// true if the following Cur() call will be valid, false otherwise. +func (s *StmtStatsIterator) Next() bool { + s.idx++ + return s.idx < len(s.stmtKeys) +} + +// Cur returns the roachpb.CollectedStatementStatistics at the current internal +// counter. +func (s *StmtStatsIterator) Cur() *roachpb.CollectedStatementStatistics { + stmtKey := s.stmtKeys[s.idx] + + stmtFingerprintID := constructStatementFingerprintIDFromStmtKey(stmtKey) + statementStats, _, _ := + s.container.getStatsForStmtWithKey(stmtKey, invalidStmtFingerprintID, false /* createIfNonexistent */) + + // If the key is not found (and we expected to find it), the table must + // have been cleared between now and the time we read all the keys. In + // that case we simply skip this key as there are no metrics to report. + if statementStats == nil { + return nil + } + + statementStats.mu.Lock() + data := statementStats.mu.data + distSQLUsed := statementStats.mu.distSQLUsed + vectorized := statementStats.mu.vectorized + fullScan := statementStats.mu.fullScan + database := statementStats.mu.database + statementStats.mu.Unlock() + + collectedStats := roachpb.CollectedStatementStatistics{ + Key: roachpb.StatementStatisticsKey{ + Query: stmtKey.anonymizedStmt, + DistSQL: distSQLUsed, + Opt: true, + Vec: vectorized, + ImplicitTxn: stmtKey.implicitTxn, + FullScan: fullScan, + Failed: stmtKey.failed, + App: s.container.appName, + Database: database, + }, + ID: stmtFingerprintID, + Stats: data, + } + + return &collectedStats +} + +// TxnStatsIterator is an iterator that iterates over the transaction statistics +// inside of a ssmemstorage.Container. +type TxnStatsIterator struct { + baseIterator + txnKeys txnList +} + +// NewTxnStatsIterator returns a new instance of TxnStatsIterator. +func NewTxnStatsIterator( + container *Container, options *sqlstats.IteratorOptions, +) *TxnStatsIterator { + var txnKeys txnList + container.mu.Lock() + for k := range container.mu.txns { + txnKeys = append(txnKeys, k) + } + container.mu.Unlock() + if options.SortedKey { + sort.Sort(txnKeys) + } + + return &TxnStatsIterator{ + baseIterator: baseIterator{ + container: container, + idx: -1, + }, + txnKeys: txnKeys, + } +} + +// Next increments the internal counter of the TxnStatsIterator. It returns +// true if the following Cur() call will be valid, false otherwise. +func (t *TxnStatsIterator) Next() bool { + t.idx++ + return t.idx < len(t.txnKeys) +} + +// Cur returns the roachpb.CollectedTransactionStatistics at the current internal +// counter. +func (t *TxnStatsIterator) Cur() ( + roachpb.TransactionFingerprintID, + *roachpb.CollectedTransactionStatistics, +) { + txnKey := t.txnKeys[t.idx] + + // We don't want to create the key if it doesn't exist, so it's okay to + // pass nil for the statementFingerprintIDs, as they are only set when a key is + // constructed. + txnStats, _, _ := t.container.getStatsForTxnWithKey(txnKey, nil /* stmtFingerprintIDs */, false /* createIfNonexistent */) + // If the key is not found (and we expected to find it), the table must + // have been cleared between now and the time we read all the keys. In + // that case we simply skip this key as there are no metrics to report. + if txnStats == nil { + return 0, nil + } + + txnStats.mu.Lock() + defer txnStats.mu.Unlock() + collectedStats := roachpb.CollectedTransactionStatistics{ + StatementFingerprintIDs: txnStats.statementFingerprintIDs, + App: t.container.appName, + Stats: txnStats.mu.data, + } + return txnKey, &collectedStats +} diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go index 3cda730e5d8b..47a0700430e9 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go @@ -18,7 +18,6 @@ import ( "context" "encoding/json" "fmt" - "sort" "sync/atomic" "unsafe" @@ -60,7 +59,8 @@ const invalidStmtFingerprintID = 0 // Container holds per-application statement and transaction statistics. type Container struct { - st *cluster.Settings + st *cluster.Settings + appName string // uniqueStmtFingerprintLimit is the limit on number of unique statement // fingerprints we can store in memory. @@ -108,9 +108,11 @@ func New( uniqueStmtFingerprintCount *int64, uniqueTxnFingerprintCount *int64, mon *mon.BytesMonitor, + appName string, ) *Container { s := &Container{ st: st, + appName: appName, uniqueStmtFingerprintLimit: uniqueStmtFingerprintLimit, uniqueTxnFingerprintLimit: uniqueTxnFingerprintLimit, } @@ -125,110 +127,6 @@ func New( return s } -// IterateStatementStats iterates through the stored statement statistics -// stored in this Container. -func (s *Container) IterateStatementStats( - ctx context.Context, appName string, orderedKey bool, visitor sqlstats.StatementVisitor, -) error { - var stmtKeys stmtList - s.mu.Lock() - for k := range s.mu.stmts { - stmtKeys = append(stmtKeys, k) - } - s.mu.Unlock() - if orderedKey { - sort.Sort(stmtKeys) - } - - for _, stmtKey := range stmtKeys { - stmtFingerprintID := constructStatementFingerprintIDFromStmtKey(stmtKey) - statementStats, _, _ := - s.getStatsForStmtWithKey(stmtKey, invalidStmtFingerprintID, false /* createIfNonexistent */) - - // If the key is not found (and we expected to find it), the table must - // have been cleared between now and the time we read all the keys. In - // that case we simply skip this key as there are no metrics to report. - if statementStats == nil { - continue - } - - statementStats.mu.Lock() - data := statementStats.mu.data - distSQLUsed := statementStats.mu.distSQLUsed - vectorized := statementStats.mu.vectorized - fullScan := statementStats.mu.fullScan - database := statementStats.mu.database - statementStats.mu.Unlock() - - collectedStats := roachpb.CollectedStatementStatistics{ - Key: roachpb.StatementStatisticsKey{ - Query: stmtKey.anonymizedStmt, - DistSQL: distSQLUsed, - Opt: true, - Vec: vectorized, - ImplicitTxn: stmtKey.implicitTxn, - FullScan: fullScan, - Failed: stmtKey.failed, - App: appName, - Database: database, - }, - ID: stmtFingerprintID, - Stats: data, - } - - err := visitor(ctx, &collectedStats) - if err != nil { - return fmt.Errorf("sql stats iteration abort: %s", err) - } - } - return nil -} - -// IterateTransactionStats iterates through the stored transaction statistics -// stored in this Container. -func (s *Container) IterateTransactionStats( - ctx context.Context, appName string, orderedKey bool, visitor sqlstats.TransactionVisitor, -) error { - // Retrieve the transaction keys and optionally sort them. - var txnKeys txnList - s.mu.Lock() - for k := range s.mu.txns { - txnKeys = append(txnKeys, k) - } - s.mu.Unlock() - if orderedKey { - sort.Sort(txnKeys) - } - - // Now retrieve the per-stmt stats proper. - for _, txnKey := range txnKeys { - // We don't want to create the key if it doesn't exist, so it's okay to - // pass nil for the statementFingerprintIDs, as they are only set when a key is - // constructed. - txnStats, _, _ := s.getStatsForTxnWithKey(txnKey, nil /* stmtFingerprintIDs */, false /* createIfNonexistent */) - // If the key is not found (and we expected to find it), the table must - // have been cleared between now and the time we read all the keys. In - // that case we simply skip this key as there are no metrics to report. - if txnStats == nil { - continue - } - - txnStats.mu.Lock() - collectedStats := roachpb.CollectedTransactionStatistics{ - StatementFingerprintIDs: txnStats.statementFingerprintIDs, - App: appName, - Stats: txnStats.mu.data, - } - txnStats.mu.Unlock() - - err := visitor(ctx, txnKey, &collectedStats) - if err != nil { - return fmt.Errorf("sql stats iteration abort: %s", err) - } - } - return nil -} - // IterateAggregatedTransactionStats iterates through the stored aggregated // transaction statistics stored in this Container. func (s *Container) IterateAggregatedTransactionStats( @@ -295,6 +193,16 @@ func (s *Container) GetTransactionStats( return collectedStats, nil } +// StmtStatsIterator returns an instance of StmtStatsIterator. +func (s *Container) StmtStatsIterator(options *sqlstats.IteratorOptions) *StmtStatsIterator { + return NewStmtStatsIterator(s, options) +} + +// TxnStatsIterator returns an instance of TxnStatsIterator. +func (s *Container) TxnStatsIterator(options *sqlstats.IteratorOptions) *TxnStatsIterator { + return NewTxnStatsIterator(s, options) +} + type txnStats struct { statementFingerprintIDs []roachpb.StmtFingerprintID