diff --git a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go index e4617952a66c..6ff3d785a704 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go @@ -790,10 +790,13 @@ func (c *tenantSideCostController) OnResponseWait( } // Record the number of RUs consumed by the IO request. - if sp := tracing.SpanFromContext(ctx); sp != nil && sp.RecordingType() != tracingpb.RecordingOff { - sp.RecordStructured(&roachpb.TenantConsumption{ - RU: float64(totalRU), - }) + if multitenant.TenantRUEstimateEnabled.Get(&c.settings.SV) { + if sp := tracing.SpanFromContext(ctx); sp != nil && + sp.RecordingType() != tracingpb.RecordingOff { + sp.RecordStructured(&roachpb.TenantConsumption{ + RU: float64(totalRU), + }) + } } c.mu.Lock() diff --git a/pkg/multitenant/BUILD.bazel b/pkg/multitenant/BUILD.bazel index 9ec11833c1d1..770cf8befb89 100644 --- a/pkg/multitenant/BUILD.bazel +++ b/pkg/multitenant/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/kv", "//pkg/multitenant/tenantcostmodel", "//pkg/roachpb", + "//pkg/settings", "//pkg/sql/sqlliveness", "//pkg/sql/sqlutil", "//pkg/util/metric", diff --git a/pkg/multitenant/cost_controller.go b/pkg/multitenant/cost_controller.go index b7ebe39c17f1..4c314e33ac30 100644 --- a/pkg/multitenant/cost_controller.go +++ b/pkg/multitenant/cost_controller.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/util/stop" ) @@ -131,3 +132,12 @@ type TenantSideExternalIORecorder interface { type exemptCtxValueType struct{} var exemptCtxValue interface{} = exemptCtxValueType{} + +// TenantRUEstimateEnabled determines whether EXPLAIN ANALYZE should return an +// estimate for the number of RUs consumed by tenants. +var TenantRUEstimateEnabled = settings.RegisterBoolSetting( + settings.TenantWritable, + "sql.tenant_ru_estimation.enabled", + "determines whether explain analyze should return an estimate for the query's RU consumption", + true, +) diff --git a/pkg/multitenant/multitenantcpu/cpu_usage.go b/pkg/multitenant/multitenantcpu/cpu_usage.go index 3c52f3d4400a..dcd2616d19dc 100644 --- a/pkg/multitenant/multitenantcpu/cpu_usage.go +++ b/pkg/multitenant/multitenantcpu/cpu_usage.go @@ -50,7 +50,7 @@ func (h *CPUUsageHelper) StartCollection( // EndCollection should be called at the end of execution for a flow in order to // get the estimated number of RUs consumed due to CPU usage. It returns zero -// for non-tenants. +// for non-tenants. It is a no-op if StartCollection was never called. func (h *CPUUsageHelper) EndCollection(ctx context.Context) (ruFomCPU float64) { if h.costController == nil || h.costController.GetCostConfig() == nil { return 0 diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index c04c6075bce2..fed11620f3f6 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/multitenantcpu" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -1090,9 +1091,11 @@ func (ex *connExecutor) dispatchToExecutionEngine( ex.sessionTracing.TracePlanStart(ctx, stmt.AST.StatementTag()) ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.PlannerStartLogicalPlan, timeutil.Now()) - if server := ex.server.cfg.DistSQLSrv; server != nil { - // Begin measuring CPU usage for tenants. This is a no-op for non-tenants. - ex.cpuStatsCollector.StartCollection(ctx, server.TenantCostController) + if multitenant.TenantRUEstimateEnabled.Get(ex.server.cfg.SV()) { + if server := ex.server.cfg.DistSQLSrv; server != nil { + // Begin measuring CPU usage for tenants. This is a no-op for non-tenants. + ex.cpuStatsCollector.StartCollection(ctx, server.TenantCostController) + } } // If adminAuditLogging is enabled, we want to check for HasAdminRole @@ -1325,7 +1328,7 @@ func populateQueryLevelStatsAndRegions( } else { // If this query is being run by a tenant, record the RUs consumed by CPU // usage and network egress to the client. - if cfg.DistSQLSrv != nil { + if multitenant.TenantRUEstimateEnabled.Get(cfg.SV()) && cfg.DistSQLSrv != nil { if costController := cfg.DistSQLSrv.TenantCostController; costController != nil { if costCfg := costController.GetCostConfig(); costCfg != nil { networkEgressRUEstimate := costCfg.PGWireEgressCost(topLevelStats.networkEgressEstimate) diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 9b50a98032f4..4710a40204c8 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -749,7 +750,8 @@ func (dsp *DistSQLPlanner) Run( recv.outputTypes = plan.GetResultTypes() recv.contendedQueryMetric = dsp.distSQLSrv.Metrics.ContendedQueriesCount - if dsp.distSQLSrv.TenantCostController != nil && planCtx.planner != nil { + if multitenant.TenantRUEstimateEnabled.Get(&dsp.st.SV) && + dsp.distSQLSrv.TenantCostController != nil && planCtx.planner != nil { if instrumentation := planCtx.planner.curPlan.instrumentation; instrumentation != nil { // Only collect the network egress estimate for a tenant that is running // EXPLAIN ANALYZE, since the overhead is non-negligible. diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index bff1ba9048c0..746afb7e7a50 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -19,6 +19,7 @@ go_library( deps = [ "//pkg/base", "//pkg/kv", + "//pkg/multitenant", "//pkg/roachpb", "//pkg/settings", "//pkg/sql/catalog/colinfo", diff --git a/pkg/sql/flowinfra/flow.go b/pkg/sql/flowinfra/flow.go index 0287b30e9963..af55c0b3de35 100644 --- a/pkg/sql/flowinfra/flow.go +++ b/pkg/sql/flowinfra/flow.go @@ -16,6 +16,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode" @@ -420,7 +421,8 @@ func (f *FlowBase) StartInternal(ctx context.Context, processors []execinfra.Pro f.status = flowRunning - if !f.Gateway && f.CollectStats { + if multitenant.TenantRUEstimateEnabled.Get(&f.Cfg.Settings.SV) && + !f.Gateway && f.CollectStats { // Remote flows begin collecting CPU usage here, and finish when the last // outbox finishes. Gateway flows are handled by the connExecutor. f.FlowCtx.TenantCPUMonitor.StartCollection(ctx, f.Cfg.TenantCostController) diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index d36798b3a687..626a5627fe5f 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -18,6 +18,7 @@ import ( "time" "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" @@ -254,7 +255,8 @@ func (ih *instrumentationHelper) Setup( ih.codec = cfg.Codec ih.origCtx = ctx ih.evalCtx = p.EvalContext() - ih.isTenant = cfg.DistSQLSrv != nil && cfg.DistSQLSrv.TenantCostController != nil + ih.isTenant = multitenant.TenantRUEstimateEnabled.Get(cfg.SV()) && cfg.DistSQLSrv != nil && + cfg.DistSQLSrv.TenantCostController != nil switch ih.outputMode { case explainAnalyzeDebugOutput: