Skip to content

Commit

Permalink
opt: refactor provided ordering building logic
Browse files Browse the repository at this point in the history
This patch separates the responsibility of constructing the provided
ordering for each operator and verifying its correctness, from that of
remapping it to the set of output columns and simplifying it to reflect
known functional dependencies. This decreases the complexity of provided
ordering construction, and unifies the logic for ensuring provided
orderings can be mapped back to their required ordering choices.

Informs cockroachdb#96288

Release note: None
  • Loading branch information
DrewKimball committed Apr 6, 2023
1 parent 368e9a9 commit 25e39ca
Show file tree
Hide file tree
Showing 23 changed files with 184 additions and 777 deletions.
4 changes: 1 addition & 3 deletions pkg/sql/opt/ordering/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,6 @@ go_test(
name = "ordering_test",
size = "small",
srcs = [
"group_by_test.go",
"inverted_join_test.go",
"lookup_join_test.go",
"ordering_test.go",
"project_test.go",
Expand All @@ -66,7 +64,7 @@ go_test(
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
)

Expand Down
6 changes: 0 additions & 6 deletions pkg/sql/opt/ordering/distribute.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
package ordering

import (
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/props"
)
Expand All @@ -27,8 +26,3 @@ func distributeBuildChildReqOrdering(
// We can pass through any required ordering to the input.
return *required
}

func distributeBuildProvided(expr memo.RelExpr, required *props.OrderingChoice) opt.Ordering {
d := expr.(*memo.DistributeExpr)
return d.Input.ProvidedPhysical().Ordering
}
9 changes: 5 additions & 4 deletions pkg/sql/opt/ordering/doc.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,9 +63,10 @@ calculated bottom-up, in conjunction with the Required ordering at the level of
each operator.
The Provided ordering is a specific opt.Ordering which describes the ordering
produced by the operator, and which intersects the Required OrderingChoice (when
the operator's FDs are taken into account). A best-effort attempt is made to
keep the Provided ordering as simple as possible, to minimize the comparisons
that are necessary to maintain it.
produced by the operator, and which intersects the Required OrderingChoice. No
additional information should be required to prove that the Provided ordering
satisfies the Required OrderingChoice (e.g. functional dependencies).
A best-effort attempt is made to keep the Provided ordering as simple as
possible, to minimize the comparisons that are necessary to maintain it.
*/
package ordering
21 changes: 0 additions & 21 deletions pkg/sql/opt/ordering/group_by.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,17 +56,6 @@ func groupByBuildChildReqOrdering(
return result
}

func groupByBuildProvided(expr memo.RelExpr, required *props.OrderingChoice) opt.Ordering {
groupBy := expr.(*memo.GroupByExpr)
provided := groupBy.Input.ProvidedPhysical().Ordering
inputFDs := &groupBy.Input.Relational().FuncDeps

// Since the input's provided ordering has to satisfy both <required> and the
// GroupBy internal ordering, it may need to be trimmed.
provided = trimProvided(provided, required, inputFDs)
return remapProvided(provided, inputFDs, groupBy.GroupingCols)
}

func distinctOnCanProvideOrdering(expr memo.RelExpr, required *props.OrderingChoice) bool {
// DistinctOn may require a certain ordering of its input, but can also pass
// through a stronger ordering on the grouping columns.
Expand All @@ -87,16 +76,6 @@ func distinctOnBuildChildReqOrdering(
return result
}

func distinctOnBuildProvided(expr memo.RelExpr, required *props.OrderingChoice) opt.Ordering {
input := expr.Child(0).(memo.RelExpr)
provided := input.ProvidedPhysical().Ordering
inputFDs := &input.Relational().FuncDeps
// Since the input's provided ordering has to satisfy both <required> and the
// DistinctOn internal ordering, it may need to be trimmed.
provided = trimProvided(provided, required, inputFDs)
return remapProvided(provided, inputFDs, expr.Relational().OutputCols)
}

// StreamingGroupingColOrdering returns an ordering on grouping columns that is
// guaranteed on the input of an aggregation operator. This ordering can be used
// to perform a streaming aggregation.
Expand Down
133 changes: 0 additions & 133 deletions pkg/sql/opt/ordering/group_by_test.go

This file was deleted.

33 changes: 0 additions & 33 deletions pkg/sql/opt/ordering/inverted_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
package ordering

import (
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/props"
)
Expand Down Expand Up @@ -43,35 +42,3 @@ func invertedJoinBuildChildReqOrdering(
// (see #36219), but it should be handled correctly here nevertheless.
return trimColumnGroups(&res, &child.Relational().FuncDeps)
}

func invertedJoinBuildProvided(expr memo.RelExpr, required *props.OrderingChoice) opt.Ordering {
invertedJoin := expr.(*memo.InvertedJoinExpr)
childProvided := invertedJoin.Input.ProvidedPhysical().Ordering

// The inverted join includes an implicit projection (invertedJoin.Cols);
// some of the input columns might not be output columns so we may need to
// remap them. First check if we need to.
needsRemap := false
for i := range childProvided {
if !invertedJoin.Cols.Contains(childProvided[i].ID()) {
needsRemap = true
break
}
}
if !needsRemap {
// Fast path: we don't need to remap any columns.
return childProvided
}

// Because of the implicit projection, the FDs of the InvertedJoin don't
// include all the columns we care about; we have to recreate the FDs of
// the join before the projection. These are the FDs of the input plus any
// equality constraints in the ON condition.
var fds props.FuncDepSet
fds.CopyFrom(&invertedJoin.Input.Relational().FuncDeps)
for i := range invertedJoin.On {
fds.AddEquivFrom(&invertedJoin.On[i].ScalarProps().FuncDeps)
}

return remapProvided(childProvided, &fds, invertedJoin.Cols)
}
Loading

0 comments on commit 25e39ca

Please sign in to comment.