Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

release-23.2.4-rc: sql: collect table statistics on virtual computed columns #120933

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
1 change: 1 addition & 0 deletions docs/generated/settings/settings-for-tenants.txt
Expand Up @@ -299,6 +299,7 @@ sql.stats.response.max integer 20000 the maximum number of statements and transa
sql.stats.response.show_internal.enabled boolean false controls if statistics for internal executions should be returned by the CombinedStatements and if internal sessions should be returned by the ListSessions endpoints. These endpoints are used to display statistics on the SQL Activity pages application
sql.stats.system_tables.enabled boolean true when true, enables use of statistics on system tables by the query optimizer application
sql.stats.system_tables_autostats.enabled boolean true when true, enables automatic collection of statistics on system tables application
sql.stats.virtual_computed_columns.enabled boolean false set to true to collect table statistics on virtual computed columns application
sql.telemetry.query_sampling.enabled boolean false when set to true, executed queries will emit an event on the telemetry logging channel application
sql.telemetry.query_sampling.internal.enabled boolean false when set to true, internal queries will be sampled in telemetry logging application
sql.telemetry.query_sampling.max_event_frequency integer 8 the max event frequency (events per second) at which we sample executions for telemetry, note that it is recommended that this value shares a log-line limit of 10 logs per second on the telemetry pipeline with all other telemetry events. If sampling mode is set to 'transaction', this value is ignored. application
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Expand Up @@ -248,6 +248,7 @@
<tr><td><div id="setting-sql-stats-response-show-internal-enabled" class="anchored"><code>sql.stats.response.show_internal.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>controls if statistics for internal executions should be returned by the CombinedStatements and if internal sessions should be returned by the ListSessions endpoints. These endpoints are used to display statistics on the SQL Activity pages</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-system-tables-enabled" class="anchored"><code>sql.stats.system_tables.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>when true, enables use of statistics on system tables by the query optimizer</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-system-tables-autostats-enabled" class="anchored"><code>sql.stats.system_tables_autostats.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>when true, enables automatic collection of statistics on system tables</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-stats-virtual-computed-columns-enabled" class="anchored"><code>sql.stats.virtual_computed_columns.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>set to true to collect table statistics on virtual computed columns</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-telemetry-query-sampling-enabled" class="anchored"><code>sql.telemetry.query_sampling.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>when set to true, executed queries will emit an event on the telemetry logging channel</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-telemetry-query-sampling-internal-enabled" class="anchored"><code>sql.telemetry.query_sampling.internal.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>when set to true, internal queries will be sampled in telemetry logging</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-telemetry-query-sampling-max-event-frequency" class="anchored"><code>sql.telemetry.query_sampling.max_event_frequency</code></div></td><td>integer</td><td><code>8</code></td><td>the max event frequency (events per second) at which we sample executions for telemetry, note that it is recommended that this value shares a log-line limit of 10 logs per second on the telemetry pipeline with all other telemetry events. If sampling mode is set to &#39;transaction&#39;, this value is ignored.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
Expand Down
131 changes: 107 additions & 24 deletions pkg/sql/create_stats.go
Expand Up @@ -61,14 +61,25 @@ var featureStatsEnabled = settings.RegisterBoolSetting(
featureflag.FeatureFlagEnabledDefault,
settings.WithPublic)

var statsOnVirtualCols = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"sql.stats.virtual_computed_columns.enabled",
"set to true to collect table statistics on virtual computed columns",
false,
settings.WithPublic)

const nonIndexColHistogramBuckets = 2

// StubTableStats generates "stub" statistics for a table which are missing
// histograms and have 0 for all values.
// statistics on virtual computed columns, multi-column stats, and histograms,
// and have 0 for all values.
func StubTableStats(
desc catalog.TableDescriptor, name string, multiColEnabled bool, defaultHistogramBuckets uint32,
desc catalog.TableDescriptor, name string,
) ([]*stats.TableStatisticProto, error) {
colStats, err := createStatsDefaultColumns(desc, multiColEnabled, defaultHistogramBuckets)
colStats, err := createStatsDefaultColumns(
context.Background(), desc, false /* virtColEnabled */, false, /* multiColEnabled */
nonIndexColHistogramBuckets, nil, /* evalCtx */
)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -232,17 +243,18 @@ func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, erro
var colStats []jobspb.CreateStatsDetails_ColStat
var deleteOtherStats bool
if len(n.ColumnNames) == 0 {
// Disable multi-column stats and deleting stats
// if partial statistics at the extremes are requested.
// TODO (faizaanmadhani): Add support for multi-column stats.
virtColEnabled := statsOnVirtualCols.Get(n.p.ExecCfg().SV())
// Disable multi-column stats and deleting stats if partial statistics at
// the extremes are requested.
// TODO(faizaanmadhani): Add support for multi-column stats.
var multiColEnabled bool
if !n.Options.UsingExtremes {
multiColEnabled = stats.MultiColumnStatisticsClusterMode.Get(&n.p.ExecCfg().Settings.SV)
multiColEnabled = stats.MultiColumnStatisticsClusterMode.Get(n.p.ExecCfg().SV())
deleteOtherStats = true
}
defaultHistogramBuckets := stats.GetDefaultHistogramBuckets(n.p.ExecCfg().SV(), tableDesc)
if colStats, err = createStatsDefaultColumns(
tableDesc, multiColEnabled, defaultHistogramBuckets,
ctx, tableDesc, virtColEnabled, multiColEnabled, defaultHistogramBuckets, n.p.EvalContext(),
); err != nil {
return nil, err
}
Expand All @@ -254,12 +266,16 @@ func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, erro

columnIDs := make([]descpb.ColumnID, len(columns))
for i := range columns {
if columns[i].IsVirtual() {
return nil, pgerror.Newf(
if columns[i].IsVirtual() && !statsOnVirtualCols.Get(n.p.ExecCfg().SV()) {
err := pgerror.Newf(
pgcode.InvalidColumnReference,
"cannot create statistics on virtual column %q",
columns[i].ColName(),
)
return nil, errors.WithHint(err,
"set cluster setting sql.stats.virtual_computed_columns.enabled to collect statistics "+
"on virtual columns",
)
}
columnIDs[i] = columns[i].GetID()
}
Expand Down Expand Up @@ -354,12 +370,47 @@ const maxNonIndexCols = 100
// other columns from the table. We only collect histograms for index columns,
// plus any other boolean or enum columns (where the "histogram" is tiny).
func createStatsDefaultColumns(
desc catalog.TableDescriptor, multiColEnabled bool, defaultHistogramBuckets uint32,
ctx context.Context,
desc catalog.TableDescriptor,
virtColEnabled, multiColEnabled bool,
defaultHistogramBuckets uint32,
evalCtx *eval.Context,
) ([]jobspb.CreateStatsDetails_ColStat, error) {
colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.ActiveIndexes()))

requestedStats := make(map[string]struct{})

// CREATE STATISTICS only runs as a fully-distributed plan. If statistics on
// virtual computed columns are enabled, we must check whether each virtual
// computed column expression is safe to distribute. Virtual computed columns
// with expressions *not* safe to distribute will be skipped, even if
// sql.stats.virtual_computed_columns.enabled is true.
// TODO(michae2): Add the ability to run CREATE STATISTICS locally if a
// local-only virtual computed column expression is needed.
cannotDistribute := make([]bool, len(desc.PublicColumns()))
if virtColEnabled {
semaCtx := tree.MakeSemaContext()
exprs, _, err := schemaexpr.MakeComputedExprs(
ctx,
desc.PublicColumns(),
desc.PublicColumns(),
desc,
tree.NewUnqualifiedTableName(tree.Name(desc.GetName())),
evalCtx,
&semaCtx,
)
if err != nil {
return nil, err
}
for i, col := range desc.PublicColumns() {
cannotDistribute[i] = col.IsVirtual() && checkExprForDistSQL(exprs[i]) != nil
}
}

isUnsupportedVirtual := func(col catalog.Column) bool {
return col.IsVirtual() && (!virtColEnabled || cannotDistribute[col.Ordinal()])
}

// sortAndTrackStatsExists adds the given column IDs as a set to the
// requestedStats set. If the columnIDs were already in the set, it returns
// true. As a side-effect sortAndTrackStatsExists also sorts colIDs. NOTE:
Expand All @@ -382,11 +433,14 @@ func createStatsDefaultColumns(
return err
}

// Do not collect stats for virtual computed columns. DistSQLPlanner
// cannot currently collect stats for these columns because it plans
// table readers on the table's primary index which does not include
// virtual computed columns.
if col.IsVirtual() {
// There shouldn't be any non-public columns, but defensively skip over them
// if there are.
if !col.Public() {
return nil
}

// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
return nil
}

Expand Down Expand Up @@ -431,9 +485,30 @@ func createStatsDefaultColumns(
continue
}

colIDs := make([]descpb.ColumnID, i+1)
colIDs := make([]descpb.ColumnID, 0, i+1)
for j := 0; j <= i; j++ {
colIDs[j] = desc.GetPrimaryIndex().GetKeyColumnID(j)
col, err := catalog.MustFindColumnByID(desc, desc.GetPrimaryIndex().GetKeyColumnID(j))
if err != nil {
return nil, err
}

// There shouldn't be any non-public columns, but defensively skip over
// them if there are.
if !col.Public() {
continue
}

// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
continue
}
colIDs = append(colIDs, col.GetID())
}

// Do not attempt to create multi-column stats with < 2 columns. This can
// happen when an index contains only virtual computed columns.
if len(colIDs) < 2 {
continue
}

// Remember the requested stats so we don't request duplicates.
Expand Down Expand Up @@ -468,15 +543,23 @@ func createStatsDefaultColumns(
if err != nil {
return nil, err
}
if col.IsVirtual() {

// There shouldn't be any non-public columns, but defensively skip them
// if there are.
if !col.Public() {
continue
}

// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
continue
}
colIDs = append(colIDs, col.GetID())
}

// Do not attempt to create multi-column stats with no columns. This
// can happen when an index contains only virtual computed columns.
if len(colIDs) == 0 {
// Do not attempt to create multi-column stats with < 2 columns. This can
// happen when an index contains only virtual computed columns.
if len(colIDs) < 2 {
continue
}

Expand Down Expand Up @@ -524,8 +607,8 @@ func createStatsDefaultColumns(
for i := 0; i < len(desc.PublicColumns()) && nonIdxCols < maxNonIndexCols; i++ {
col := desc.PublicColumns()[i]

// Do not collect stats for virtual computed columns.
if col.IsVirtual() {
// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
continue
}

Expand Down
26 changes: 13 additions & 13 deletions pkg/sql/distsql_physical_planner.go
Expand Up @@ -387,9 +387,9 @@ func hasOidType(t *types.T) bool {
return false
}

// checkExpr verifies that an expression doesn't contain things that are not yet
// supported by distSQL, like distSQL-blocklisted functions.
func checkExpr(expr tree.Expr) error {
// checkExprForDistSQL verifies that an expression doesn't contain things that
// are not yet supported by distSQL, like distSQL-blocklisted functions.
func checkExprForDistSQL(expr tree.Expr) error {
if expr == nil {
return nil
}
Expand Down Expand Up @@ -536,7 +536,7 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) {
return checkSupportForPlanNode(n.source)

case *filterNode:
if err := checkExpr(n.filter); err != nil {
if err := checkExprForDistSQL(n.filter); err != nil {
return cannotDistribute, err
}
return checkSupportForPlanNode(n.source.plan)
Expand Down Expand Up @@ -575,7 +575,7 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) {
// TODO(nvanbenschoten): lift this restriction.
return cannotDistribute, cannotDistributeRowLevelLockingErr
}
if err := checkExpr(n.onExpr); err != nil {
if err := checkExprForDistSQL(n.onExpr); err != nil {
return cannotDistribute, err
}
rec, err := checkSupportForPlanNode(n.input)
Expand All @@ -585,7 +585,7 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) {
return rec.compose(shouldDistribute), nil

case *joinNode:
if err := checkExpr(n.pred.onCond); err != nil {
if err := checkExprForDistSQL(n.pred.onCond); err != nil {
return cannotDistribute, err
}
recLeft, err := checkSupportForPlanNode(n.left.plan)
Expand Down Expand Up @@ -624,13 +624,13 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) {
return cannotDistribute, cannotDistributeRowLevelLockingErr
}

if err := checkExpr(n.lookupExpr); err != nil {
if err := checkExprForDistSQL(n.lookupExpr); err != nil {
return cannotDistribute, err
}
if err := checkExpr(n.remoteLookupExpr); err != nil {
if err := checkExprForDistSQL(n.remoteLookupExpr); err != nil {
return cannotDistribute, err
}
if err := checkExpr(n.onCond); err != nil {
if err := checkExprForDistSQL(n.onCond); err != nil {
return cannotDistribute, err
}
rec, err := checkSupportForPlanNode(n.input)
Expand All @@ -646,15 +646,15 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) {

case *projectSetNode:
for i := range n.exprs {
if err := checkExpr(n.exprs[i]); err != nil {
if err := checkExprForDistSQL(n.exprs[i]); err != nil {
return cannotDistribute, err
}
}
return checkSupportForPlanNode(n.source)

case *renderNode:
for _, e := range n.render {
if err := checkExpr(e); err != nil {
if err := checkExprForDistSQL(e); err != nil {
return cannotDistribute, err
}
}
Expand Down Expand Up @@ -725,7 +725,7 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) {

for _, tuple := range n.tuples {
for _, expr := range tuple {
if err := checkExpr(expr); err != nil {
if err := checkExprForDistSQL(expr); err != nil {
return cannotDistribute, err
}
}
Expand Down Expand Up @@ -759,7 +759,7 @@ func checkSupportForPlanNode(node planNode) (distRecommendation, error) {
return cannotDistribute, cannotDistributeRowLevelLockingErr
}
}
if err := checkExpr(n.onCond); err != nil {
if err := checkExprForDistSQL(n.onCond); err != nil {
return cannotDistribute, err
}
return shouldDistribute, nil
Expand Down