From c12c4d3753d9e44e2536ca348393d0835c600d22 Mon Sep 17 00:00:00 2001 From: Barry He Date: Wed, 27 Jan 2021 15:04:34 -0800 Subject: [PATCH] sql: add full table or index scan count to metrics This new metric allows for users to see a time series of their full table or index scans in the advanced debug console. This metric is part of EngineMetrics, so there's a corresponding internal metric that counts internal full table or index scans from internal engine queries. Release note (ui change): User can see time series of full table or index scans in advanced debug console. --- pkg/sql/conn_executor.go | 8 ++++---- pkg/sql/conn_executor_exec.go | 24 +++++++++++++----------- pkg/sql/exec_util.go | 7 ++++++- pkg/sql/executor_statement_metrics.go | 3 +++ 4 files changed, 26 insertions(+), 16 deletions(-) diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 66446b9ff8e1..6167a42071df 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -301,7 +302,6 @@ func makeMetrics(internal bool) Metrics { SQLOptFallbackCount: metric.NewCounter(getMetricMeta(MetaSQLOptFallback, internal)), SQLOptPlanCacheHits: metric.NewCounter(getMetricMeta(MetaSQLOptPlanCacheHits, internal)), SQLOptPlanCacheMisses: metric.NewCounter(getMetricMeta(MetaSQLOptPlanCacheMisses, internal)), - // TODO(mrtracy): See HistogramWindowInterval in server/config.go for the 6x factor. DistSQLExecLatency: metric.NewLatency(getMetricMeta(MetaDistSQLExecLatency, internal), 6*metricsSampleInterval), @@ -315,8 +315,9 @@ func makeMetrics(internal bool) Metrics { 6*metricsSampleInterval), SQLTxnsOpen: metric.NewGauge(getMetricMeta(MetaSQLTxnsOpen, internal)), - TxnAbortCount: metric.NewCounter(getMetricMeta(MetaTxnAbort, internal)), - FailureCount: metric.NewCounter(getMetricMeta(MetaFailure, internal)), + TxnAbortCount: metric.NewCounter(getMetricMeta(MetaTxnAbort, internal)), + FailureCount: metric.NewCounter(getMetricMeta(MetaFailure, internal)), + FullTableOrIndexScanCount: metric.NewCounter(getMetricMeta(MetaFullTableOrIndexScan, internal)), }, StartedStatementCounters: makeStartedStatementCounters(internal), ExecutedStatementCounters: makeExecutedStatementCounters(internal), @@ -566,7 +567,6 @@ func (s *Server) newConnExecutor( nodeIDOrZero, _ := s.cfg.NodeID.OptionalNodeID() sdMutator := new(sessionDataMutator) *sdMutator = s.makeSessionDataMutator(sd, sdDefaults) - ex := &connExecutor{ server: s, metrics: srvMetrics, diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index d9f63af3265a..649958041bb2 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -914,17 +914,19 @@ func (ex *connExecutor) makeExecPlan(ctx context.Context, planner *planner) erro flags := planner.curPlan.flags - // We don't execute the statement if: - // - plan contains a full table or full index scan. - // - the session setting disallows full table/index scans. - // - the query is not an internal query. - if (flags.IsSet(planFlagContainsFullIndexScan) || flags.IsSet(planFlagContainsFullTableScan)) && - planner.EvalContext().SessionData.DisallowFullTableScans && ex.executorType == executorTypeExec { - return errors.WithHint( - pgerror.Newf(pgcode.TooManyRows, - "query `%s` contains a full table/index scan which is explicitly disallowed", - planner.stmt.SQL), - "try overriding the `disallow_full_table_scans` cluster/session setting") + if flags.IsSet(planFlagContainsFullIndexScan) || flags.IsSet(planFlagContainsFullTableScan) { + if ex.executorType == executorTypeExec && planner.EvalContext().SessionData.DisallowFullTableScans { + // We don't execute the statement if: + // - plan contains a full table or full index scan. + // - the session setting disallows full table/index scans. + // - the query is not an internal query. + return errors.WithHint( + pgerror.Newf(pgcode.TooManyRows, + "query `%s` contains a full table/index scan which is explicitly disallowed", + planner.stmt.SQL), + "try overriding the `disallow_full_table_scans` cluster/session setting") + } + ex.metrics.EngineMetrics.FullTableOrIndexScanCount.Inc(1) } // TODO(knz): Remove this accounting if/when savepoint rollbacks diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 20bf1e0412a3..d6aa11f2ae21 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -488,9 +488,14 @@ var ( Measurement: "Open SQL Transactions", Unit: metric.Unit_COUNT, } + MetaFullTableOrIndexScan = metric.Metadata{ + Name: "sql.full.scan.count", + Help: "Number of full table or index scans", + Measurement: "SQL Statements", + Unit: metric.Unit_COUNT, + } // Below are the metadata for the statement started counters. - MetaQueryStarted = metric.Metadata{ Name: "sql.query.started.count", Help: "Number of SQL queries started", diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index e76c2da419e8..43297aac3249 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -140,6 +140,9 @@ type EngineMetrics struct { // FailureCount counts non-retriable errors in open transactions. FailureCount *metric.Counter + + // FullTableOrIndexScanCount counts the number of full table or index scans. + FullTableOrIndexScanCount *metric.Counter } // EngineMetrics implements the metric.Struct interface