Skip to content

Commit

Permalink
sql: Make USING EXTREMES predicates parsable
Browse files Browse the repository at this point in the history
Previously the USING EXTREMES predicates
were constructed as strings which would
make them very difficult to parse. This PR
changes the way these predicates are
constructed so that they can later be parsed.

Release note (sql change): USING EXTREME predicates in the
output of SHOW STATISTICS will have additional
parenthesis and type assertions.

Epic: CRDB-19449
  • Loading branch information
faizaanmadhani committed Dec 6, 2022
1 parent db71e04 commit 1b50af0
Show file tree
Hide file tree
Showing 2 changed files with 71 additions and 45 deletions.
84 changes: 55 additions & 29 deletions pkg/sql/distsql_plan_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package sql

import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/jobs"
Expand All @@ -29,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp"
"github.com/cockroachdb/cockroach/pkg/sql/span"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -236,12 +236,15 @@ func (dsp *DistSQLPlanner) createPartialStatsPlan(
"or the prior histogram has no buckets and a partial statistic cannot be collected",
column.GetName())
}

extremesSpans, err := constructUsingExtremesSpans(planCtx, histogram, scan.index)
lowerBound, upperBound, err := getUsingExtremesBounds(planCtx, histogram)
if err != nil {
return nil, err
}
extremesPredicate := constructUsingExtremesPredicate(extremesSpans, column.GetName(), scan.index)
extremesSpans, err := constructUsingExtremesSpans(lowerBound, upperBound, scan.index)
if err != nil {
return nil, err
}
extremesPredicate := constructUsingExtremesPredicate(lowerBound, upperBound, column.GetName(), scan.index)
// Get roachpb.Spans from constraint.Spans
scan.spans, err = sb.SpansFromConstraintSpan(&extremesSpans, span.NoopSplitter())
if err != nil {
Expand Down Expand Up @@ -505,29 +508,13 @@ func (dsp *DistSQLPlanner) planAndRunCreateStats(
return resultWriter.Err()
}

// constructUsingExtremesPredicate returns string of a predicate identifying
// the upper and lower bounds of the stats collection.
func constructUsingExtremesPredicate(
extremesSpans constraint.Spans, columnName string, index catalog.Index,
) string {
var predicate string
if index.GetKeyColumnDirection(0) == catpb.IndexColumn_ASC {
predicate = fmt.Sprintf("%s < %s OR %s > %s OR %s is NULL", columnName, extremesSpans.Get(0).EndKey().Value(0).String(), columnName, extremesSpans.Get(1).StartKey().Value(0).String(), columnName)
} else {
predicate = fmt.Sprintf("%s < %s OR %s > %s OR %s is NULL", columnName, extremesSpans.Get(1).StartKey().Value(0).String(), columnName, extremesSpans.Get(0).EndKey().Value(0).String(), columnName)
}
return predicate
}

// constructExtremesSpans returns a constraint.Spans consisting of a
// lowerbound and upperbound span covering the extremes of an index.
func constructUsingExtremesSpans(
planCtx *PlanningCtx, histogram []cat.HistogramBucket, index catalog.Index,
) (constraint.Spans, error) {
var lbSpan constraint.Span
var ubSpan constraint.Span
// getUsingExtremesBounds returns a tree.Datum representing the upper and lower
// bounds of the USING EXTREMES span for partial statistics.
func getUsingExtremesBounds(
planCtx *PlanningCtx, histogram []cat.HistogramBucket,
) (tree.Datum, tree.Datum, error) {
lowerBound := histogram[0].UpperBound
upperBound := constraint.MakeKey(histogram[len(histogram)-1].UpperBound)
upperBound := histogram[len(histogram)-1].UpperBound
// Pick the earliest lowerBound that is not null,
// but if none exist, return error
for i := range histogram {
Expand All @@ -538,18 +525,57 @@ func constructUsingExtremesSpans(
}
}
if lowerBound.Compare(planCtx.EvalContext(), tree.DNull) == 0 {
return constraint.Spans{},
return tree.DNull, tree.DNull,
pgerror.Newf(
pgcode.ObjectNotInPrerequisiteState,
"only NULL values exist in the index, so partial stats cannot be collected")
}
return lowerBound, upperBound, nil
}

// constructUsingExtremesPredicate returns string of a predicate identifying
// the upper and lower bounds of the stats collection.
func constructUsingExtremesPredicate(
lowerBound tree.Datum, upperBound tree.Datum, columnName string, index catalog.Index,
) string {
lbExpr := tree.ComparisonExpr{
Operator: treecmp.MakeComparisonOperator(treecmp.LT),
Left: &tree.ColumnItem{ColumnName: tree.Name(columnName)},
Right: lowerBound,
}

ubExpr := tree.ComparisonExpr{
Operator: treecmp.MakeComparisonOperator(treecmp.GT),
Left: &tree.ColumnItem{ColumnName: tree.Name(columnName)},
Right: upperBound,
}
nullExpr := tree.IsNullExpr{
Expr: &tree.ColumnItem{ColumnName: tree.Name(columnName)},
}

pred := tree.OrExpr{
Left: &lbExpr,
Right: &tree.OrExpr{
Left: &ubExpr,
Right: &nullExpr,
},
}
return tree.Serialize(&pred)
}

// constructExtremesSpans returns a constraint.Spans consisting of a
// lowerbound and upperbound span covering the extremes of an index.
func constructUsingExtremesSpans(
lowerBound tree.Datum, upperBound tree.Datum, index catalog.Index,
) (constraint.Spans, error) {
var lbSpan constraint.Span
var ubSpan constraint.Span
if index.GetKeyColumnDirection(0) == catpb.IndexColumn_ASC {
lbSpan.Init(constraint.EmptyKey, constraint.IncludeBoundary, constraint.MakeKey(lowerBound), constraint.ExcludeBoundary)
ubSpan.Init(upperBound, constraint.ExcludeBoundary, constraint.EmptyKey, constraint.IncludeBoundary)
ubSpan.Init(constraint.MakeKey(upperBound), constraint.ExcludeBoundary, constraint.EmptyKey, constraint.IncludeBoundary)
} else {
lbSpan.Init(constraint.MakeKey(lowerBound), constraint.ExcludeBoundary, constraint.EmptyKey, constraint.IncludeBoundary)
ubSpan.Init(constraint.EmptyKey, constraint.IncludeBoundary, upperBound, constraint.ExcludeBoundary)
ubSpan.Init(constraint.EmptyKey, constraint.IncludeBoundary, constraint.MakeKey(upperBound), constraint.ExcludeBoundary)
}
var extremesSpans constraint.Spans
if index.GetKeyColumnDirection(0) == catpb.IndexColumn_ASC {
Expand Down
32 changes: 16 additions & 16 deletions pkg/sql/logictest/testdata/logic_test/distsql_stats
Original file line number Diff line number Diff line change
Expand Up @@ -2044,19 +2044,19 @@ query TT colnames
SELECT "name", "partialPredicate" FROM system.table_statistics WHERE name='abcd_a_partial';
----
name partialPredicate
abcd_a_partial a < 1 OR a > 8 OR a is NULL
abcd_a_partial (a < 1:::INT8) OR ((a > 8:::INT8) OR (a IS NULL))

query TT colnames
SELECT "name", "partialPredicate" FROM system.table_statistics WHERE name='abcd_c_partial';
----
name partialPredicate
abcd_c_partial c < 100 OR c > 800 OR c is NULL
abcd_c_partial (c < 100:::INT8) OR ((c > 800:::INT8) OR (c IS NULL))

query TT colnames
SELECT "name", "partialPredicate" FROM system.table_statistics WHERE name='xy_x_partial';
----
name partialPredicate
xy_x_partial x < 0 OR x > 3 OR x is NULL
xy_x_partial (x < 0:::INT8) OR ((x > 3:::INT8) OR (x IS NULL))

# Test if requesting a partial stat again uses the previous full stat and not the previous partial stat.
statement ok
Expand All @@ -2065,10 +2065,10 @@ CREATE STATISTICS xy_x_partial_2 ON x FROM xy USING EXTREMES
query TTII colnames
SELECT "statistics_name", "partial_predicate", "row_count", "null_count" FROM [SHOW STATISTICS FOR TABLE xy];
----
statistics_name partial_predicate row_count null_count
xy_x NULL 4 0
xy_x_partial x < 0 OR x > 3 OR x is NULL 4 0
xy_x_partial_2 x < 0 OR x > 3 OR x is NULL 4 0
statistics_name partial_predicate row_count null_count
xy_x NULL 4 0
xy_x_partial (x < 0:::INT8) OR ((x > 3:::INT8) OR (x IS NULL)) 4 0
xy_x_partial_2 (x < 0:::INT8) OR ((x > 3:::INT8) OR (x IS NULL)) 4 0

# Test null values.
statement ok
Expand Down Expand Up @@ -2107,9 +2107,9 @@ upper_bound range_rows distinct_range_rows equal_rows
query TTII colnames
SELECT "statistics_name", "partial_predicate", "row_count", "null_count" FROM [SHOW STATISTICS FOR TABLE a_null];
----
statistics_name partial_predicate row_count null_count
a_null_stat NULL 3 1
a_null_stat_partial a < 1 OR a > 2 OR a is NULL 4 4
statistics_name partial_predicate row_count null_count
a_null_stat NULL 3 1
a_null_stat_partial (a < 1:::INT8) OR ((a > 2:::INT8) OR (a IS NULL)) 4 4

# Test descending indexes.
statement ok
Expand Down Expand Up @@ -2150,9 +2150,9 @@ upper_bound range_rows distinct_range_rows equal_rows
query TTII colnames
SELECT "statistics_name", "partial_predicate", "row_count", "null_count" FROM [SHOW STATISTICS FOR TABLE d_desc];
----
statistics_name partial_predicate row_count null_count
sd NULL 4 0
sdp a < 1 OR a > 4 OR a is NULL 2 0
statistics_name partial_predicate row_count null_count
sd NULL 4 0
sdp (a < 1:::INT8) OR ((a > 4:::INT8) OR (a IS NULL)) 2 0

# Test descending index with NULL
statement ok
Expand All @@ -2177,9 +2177,9 @@ CREATE STATISTICS sdnp ON a FROM d_desc USING EXTREMES;
query TTII colnames
SELECT "statistics_name", "partial_predicate", "row_count", "null_count" FROM [SHOW STATISTICS FOR TABLE d_desc];
----
statistics_name partial_predicate row_count null_count
sdn NULL 8 2
sdnp a < 0 OR a > 5 OR a is NULL 2 2
statistics_name partial_predicate row_count null_count
sdn NULL 8 2
sdnp (a < 0:::INT8) OR ((a > 5:::INT8) OR (a IS NULL)) 2 2

# Verify errors.
statement error pq: cannot process multiple partial statistics at once
Expand Down

0 comments on commit 1b50af0

Please sign in to comment.