Skip to content

Commit

Permalink
treewindow: extract some window constants into a new package out of tree
Browse files Browse the repository at this point in the history
This commit extracts some constants about the window functions into
a new package which allows us to finally break the dependency of
`execgen` on `tree`. Now the absence of such dependency is enforced with
`VerifyNoImports` in `colexec` package.

The only notable change that I had to make is making
`WindowFrameExclusion` not implement `tree.NodeFormatter` interface;
instead, the callsites that wanted to call `Format` on that type now
call `String` which provides the same behavior.

Release note: None
  • Loading branch information
yuzefovich committed Feb 10, 2022
1 parent 2586fcd commit 0caa03d
Show file tree
Hide file tree
Showing 36 changed files with 542 additions and 475 deletions.
1 change: 1 addition & 0 deletions pkg/internal/sqlsmith/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/sem/tree/treebin",
"//pkg/sql/sem/tree/treecmp",
"//pkg/sql/sem/tree/treewindow",
"//pkg/sql/types",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
Expand Down
39 changes: 20 additions & 19 deletions pkg/internal/sqlsmith/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treewindow"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

Expand Down Expand Up @@ -457,21 +458,21 @@ func makeFunc(s *Smither, ctx Context, typ *types.T, refs colRefs) (tree.TypedEx
), typ), true
}

var windowFrameModes = []tree.WindowFrameMode{
tree.RANGE,
tree.ROWS,
tree.GROUPS,
var windowFrameModes = []treewindow.WindowFrameMode{
treewindow.RANGE,
treewindow.ROWS,
treewindow.GROUPS,
}

func randWindowFrameMode(s *Smither) tree.WindowFrameMode {
func randWindowFrameMode(s *Smither) treewindow.WindowFrameMode {
return windowFrameModes[s.rnd.Intn(len(windowFrameModes))]
}

func makeWindowFrame(s *Smither, refs colRefs, orderTypes []*types.T) *tree.WindowFrame {
var frameMode tree.WindowFrameMode
var frameMode treewindow.WindowFrameMode
for {
frameMode = randWindowFrameMode(s)
if len(orderTypes) > 0 || frameMode != tree.GROUPS {
if len(orderTypes) > 0 || frameMode != treewindow.GROUPS {
// GROUPS mode requires an ORDER BY clause, so if it is not present and
// GROUPS mode was randomly chosen, we need to generate again; otherwise,
// we're done.
Expand All @@ -494,51 +495,51 @@ func makeWindowFrame(s *Smither, refs colRefs, orderTypes []*types.T) *tree.Wind
allowRangeWithOffsets = true
}
}
if frameMode == tree.RANGE && !allowRangeWithOffsets {
if frameMode == treewindow.RANGE && !allowRangeWithOffsets {
if s.coin() {
startBound.BoundType = tree.UnboundedPreceding
startBound.BoundType = treewindow.UnboundedPreceding
} else {
startBound.BoundType = tree.CurrentRow
startBound.BoundType = treewindow.CurrentRow
}
if s.coin() {
endBound = new(tree.WindowFrameBound)
if s.coin() {
endBound.BoundType = tree.CurrentRow
endBound.BoundType = treewindow.CurrentRow
} else {
endBound.BoundType = tree.UnboundedFollowing
endBound.BoundType = treewindow.UnboundedFollowing
}
}
} else {
// There are 5 bound types, but only 4 can be used for the start bound.
startBound.BoundType = tree.WindowFrameBoundType(s.rnd.Intn(4))
if startBound.BoundType == tree.OffsetFollowing {
startBound.BoundType = treewindow.WindowFrameBoundType(s.rnd.Intn(4))
if startBound.BoundType == treewindow.OffsetFollowing {
// With OffsetFollowing as the start bound, the end bound must be
// present and can either be OffsetFollowing or UnboundedFollowing.
endBound = new(tree.WindowFrameBound)
if s.coin() {
endBound.BoundType = tree.OffsetFollowing
endBound.BoundType = treewindow.OffsetFollowing
} else {
endBound.BoundType = tree.UnboundedFollowing
endBound.BoundType = treewindow.UnboundedFollowing
}
}
if endBound == nil && s.coin() {
endBound = new(tree.WindowFrameBound)
// endBound cannot be "smaller" than startBound, so we will "prohibit" all
// such choices.
endBoundProhibitedChoices := int(startBound.BoundType)
if startBound.BoundType == tree.UnboundedPreceding {
if startBound.BoundType == treewindow.UnboundedPreceding {
// endBound cannot be UnboundedPreceding, so we always need to skip that
// choice.
endBoundProhibitedChoices = 1
}
endBound.BoundType = tree.WindowFrameBoundType(endBoundProhibitedChoices + s.rnd.Intn(5-endBoundProhibitedChoices))
endBound.BoundType = treewindow.WindowFrameBoundType(endBoundProhibitedChoices + s.rnd.Intn(5-endBoundProhibitedChoices))
}
// We will set offsets regardless of the bound type, but they will only be
// used when a bound is either OffsetPreceding or OffsetFollowing. Both
// ROWS and GROUPS mode need non-negative integers as bounds whereas RANGE
// mode takes the type as the single ORDER BY clause has.
typ := types.Int
if frameMode == tree.RANGE {
if frameMode == treewindow.RANGE {
typ = orderTypes[0]
}
startBound.OffsetExpr = makeScalar(s, typ, refs)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -369,6 +369,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/sem/tree/treebin",
"//pkg/sql/sem/tree/treecmp",
"//pkg/sql/sem/tree/treewindow",
"//pkg/sql/sessiondata",
"//pkg/sql/sessiondatapb",
"//pkg/sql/sessioninit",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/colexec/colexecwindow/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ go_test(
"//pkg/sql/execinfrapb",
"//pkg/sql/randgen",
"//pkg/sql/sem/tree",
"//pkg/sql/sem/tree/treewindow",
"//pkg/sql/types",
"//pkg/testutils/buildutil",
"//pkg/testutils/colcontainerutils",
Expand Down
87 changes: 47 additions & 40 deletions pkg/sql/colexec/colexecwindow/window_framer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treewindow"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -80,17 +81,17 @@ func TestWindowFramer(t *testing.T) {
types.Float, types.Decimal, types.Interval, types.TimestampTZ, types.Date, types.TimeTZ,
}
const randExcludeProbability = 0.5
var randExclusions = []tree.WindowFrameExclusion{
tree.ExcludeCurrentRow, tree.ExcludeGroup, tree.ExcludeTies,
var randExclusions = []treewindow.WindowFrameExclusion{
treewindow.ExcludeCurrentRow, treewindow.ExcludeGroup, treewindow.ExcludeTies,
}
for _, mode := range []tree.WindowFrameMode{tree.ROWS, tree.GROUPS, tree.RANGE} {
for _, mode := range []treewindow.WindowFrameMode{treewindow.ROWS, treewindow.GROUPS, treewindow.RANGE} {
testCfg.mode = mode
t.Run(fmt.Sprintf("mode=%v", mode.Name()), func(t *testing.T) {
for _, bound := range []tree.WindowFrameBoundType{
tree.UnboundedPreceding, tree.OffsetPreceding, tree.CurrentRow,
tree.OffsetFollowing, tree.UnboundedFollowing,
for _, bound := range []treewindow.WindowFrameBoundType{
treewindow.UnboundedPreceding, treewindow.OffsetPreceding, treewindow.CurrentRow,
treewindow.OffsetFollowing, treewindow.UnboundedFollowing,
} {
for _, bounds := range [][2]tree.WindowFrameBoundType{
for _, bounds := range [][2]treewindow.WindowFrameBoundType{
{bound, getEndBound(rng, bound)},
{getStartBound(rng, bound), bound},
} {
Expand All @@ -109,7 +110,7 @@ func TestWindowFramer(t *testing.T) {
typ = randTypes[rng.Intn(len(randTypes))]
}
testCfg.typ = typ
exclusion := tree.NoExclusion
exclusion := treewindow.NoExclusion
if rng.Float64() < randExcludeProbability {
exclusion = randExclusions[rng.Intn(len(randExclusions))]
}
Expand All @@ -123,7 +124,7 @@ func TestWindowFramer(t *testing.T) {
testWindowFramer(t, testCfg)
})
}
if mode == tree.ROWS {
if mode == treewindow.ROWS {
// An ORDER BY clause is required for RANGE and GROUPS modes.
testCfg.ordered = false
t.Run(
Expand Down Expand Up @@ -151,10 +152,10 @@ type testConfig struct {
count int
ordered bool
asc bool
mode tree.WindowFrameMode
startBound tree.WindowFrameBoundType
endBound tree.WindowFrameBoundType
exclusion tree.WindowFrameExclusion
mode treewindow.WindowFrameMode
startBound treewindow.WindowFrameBoundType
endBound treewindow.WindowFrameBoundType
exclusion treewindow.WindowFrameExclusion
memLimit int64
}

Expand Down Expand Up @@ -212,17 +213,19 @@ func testWindowFramer(t *testing.T, testCfg *testConfig) {
partition.Close(testCfg.evalCtx.Ctx())
}

func validForStart(bound tree.WindowFrameBoundType) bool {
return bound != tree.UnboundedFollowing
func validForStart(bound treewindow.WindowFrameBoundType) bool {
return bound != treewindow.UnboundedFollowing
}

func validForEnd(bound tree.WindowFrameBoundType) bool {
return bound != tree.UnboundedPreceding
func validForEnd(bound treewindow.WindowFrameBoundType) bool {
return bound != treewindow.UnboundedPreceding
}

func getStartBound(rng *rand.Rand, endBound tree.WindowFrameBoundType) tree.WindowFrameBoundType {
startBoundTypes := []tree.WindowFrameBoundType{
tree.UnboundedPreceding, tree.OffsetPreceding, tree.CurrentRow, tree.OffsetFollowing,
func getStartBound(
rng *rand.Rand, endBound treewindow.WindowFrameBoundType,
) treewindow.WindowFrameBoundType {
startBoundTypes := []treewindow.WindowFrameBoundType{
treewindow.UnboundedPreceding, treewindow.OffsetPreceding, treewindow.CurrentRow, treewindow.OffsetFollowing,
}
for {
startBound := startBoundTypes[rng.Intn(len(startBoundTypes))]
Expand All @@ -232,9 +235,11 @@ func getStartBound(rng *rand.Rand, endBound tree.WindowFrameBoundType) tree.Wind
}
}

func getEndBound(rng *rand.Rand, startBound tree.WindowFrameBoundType) tree.WindowFrameBoundType {
endBoundTypes := []tree.WindowFrameBoundType{
tree.OffsetPreceding, tree.CurrentRow, tree.OffsetFollowing, tree.UnboundedFollowing,
func getEndBound(
rng *rand.Rand, startBound treewindow.WindowFrameBoundType,
) treewindow.WindowFrameBoundType {
endBoundTypes := []treewindow.WindowFrameBoundType{
treewindow.OffsetPreceding, treewindow.CurrentRow, treewindow.OffsetFollowing, treewindow.UnboundedFollowing,
}
for {
endBound := endBoundTypes[rng.Intn(len(endBoundTypes))]
Expand Down Expand Up @@ -336,7 +341,7 @@ func initWindowFramers(
t *testing.T, testCfg *testConfig,
) (windowFramer, *tree.WindowFrameRun, *colexecutils.SpillingBuffer) {
offsetType := types.Int
if testCfg.mode == tree.RANGE {
if testCfg.mode == treewindow.RANGE {
offsetType = GetOffsetTypeFromOrderColType(t, testCfg.typ)
}
startOffset := colexectestutils.MakeRandWindowFrameRangeOffset(t, testCfg.rng, offsetType)
Expand Down Expand Up @@ -375,7 +380,7 @@ func initWindowFramers(
},
Exclusion: exclusionToExecinfrapb(testCfg.exclusion),
}
if testCfg.mode != tree.RANGE {
if testCfg.mode != treewindow.RANGE {
frame.Bounds.Start.IntOffset = uint64(*(startOffset.(*tree.DInt)))
frame.Bounds.End.IntOffset = uint64(*(endOffset.(*tree.DInt)))
}
Expand Down Expand Up @@ -480,45 +485,47 @@ func (c *peerGroupChecker) InSameGroup(i, j int) (bool, error) {
return c.partition[i].Compare(&c.evalCtx, c.partition[j]) == 0, nil
}

func modeToExecinfrapb(mode tree.WindowFrameMode) execinfrapb.WindowerSpec_Frame_Mode {
func modeToExecinfrapb(mode treewindow.WindowFrameMode) execinfrapb.WindowerSpec_Frame_Mode {
switch mode {
case tree.RANGE:
case treewindow.RANGE:
return execinfrapb.WindowerSpec_Frame_RANGE
case tree.ROWS:
case treewindow.ROWS:
return execinfrapb.WindowerSpec_Frame_ROWS
case tree.GROUPS:
case treewindow.GROUPS:
return execinfrapb.WindowerSpec_Frame_GROUPS
}
return 0
}

func boundToExecinfrapb(bound tree.WindowFrameBoundType) execinfrapb.WindowerSpec_Frame_BoundType {
func boundToExecinfrapb(
bound treewindow.WindowFrameBoundType,
) execinfrapb.WindowerSpec_Frame_BoundType {
switch bound {
case tree.UnboundedPreceding:
case treewindow.UnboundedPreceding:
return execinfrapb.WindowerSpec_Frame_UNBOUNDED_PRECEDING
case tree.OffsetPreceding:
case treewindow.OffsetPreceding:
return execinfrapb.WindowerSpec_Frame_OFFSET_PRECEDING
case tree.CurrentRow:
case treewindow.CurrentRow:
return execinfrapb.WindowerSpec_Frame_CURRENT_ROW
case tree.OffsetFollowing:
case treewindow.OffsetFollowing:
return execinfrapb.WindowerSpec_Frame_OFFSET_FOLLOWING
case tree.UnboundedFollowing:
case treewindow.UnboundedFollowing:
return execinfrapb.WindowerSpec_Frame_UNBOUNDED_FOLLOWING
}
return 0
}

func exclusionToExecinfrapb(
exclusion tree.WindowFrameExclusion,
exclusion treewindow.WindowFrameExclusion,
) execinfrapb.WindowerSpec_Frame_Exclusion {
switch exclusion {
case tree.NoExclusion:
case treewindow.NoExclusion:
return execinfrapb.WindowerSpec_Frame_NO_EXCLUSION
case tree.ExcludeCurrentRow:
case treewindow.ExcludeCurrentRow:
return execinfrapb.WindowerSpec_Frame_EXCLUDE_CURRENT_ROW
case tree.ExcludeGroup:
case treewindow.ExcludeGroup:
return execinfrapb.WindowerSpec_Frame_EXCLUDE_GROUP
case tree.ExcludeTies:
case treewindow.ExcludeTies:
return execinfrapb.WindowerSpec_Frame_EXCLUDE_TIES
}
return 0
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/colexec/dep_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ func TestNoLinkForbidden(t *testing.T) {
[]string{
"github.com/cockroachdb/cockroach/pkg/sql/catalog",
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb",
"github.com/cockroachdb/cockroach/pkg/sql/tree",
}, nil,
)
}
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -70,9 +70,9 @@ go_library(
"//pkg/col/typeconv",
"//pkg/sql/colexec/execgen",
"//pkg/sql/colexecerror",
"//pkg/sql/sem/tree",
"//pkg/sql/sem/tree/treebin",
"//pkg/sql/sem/tree/treecmp",
"//pkg/sql/sem/tree/treewindow",
"//pkg/sql/types",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_gostdlib//x/tools/imports",
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/colexec/execgen/cmd/execgen/range_offset_handler_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,9 @@ import (

"github.com/cockroachdb/cockroach/pkg/col/typeconv"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treewindow"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -68,7 +68,7 @@ func rangeOffsetHandlerGenerator(inputFileContents string, wr io.Writer) error {
}

var rangeOffsetHandlerTmplInfos []windowFrameOffsetBoundInfo
for _, bound := range []tree.WindowFrameBoundType{tree.OffsetPreceding, tree.OffsetFollowing} {
for _, bound := range []treewindow.WindowFrameBoundType{treewindow.OffsetPreceding, treewindow.OffsetFollowing} {
boundInfo := windowFrameOffsetBoundInfo{BoundType: bound}
for _, isStart := range []bool{true, false} {
isStartInfo := windowFrameOffsetIsStartInfo{IsStart: isStart}
Expand Down Expand Up @@ -124,7 +124,7 @@ func init() {
}

type windowFrameOffsetBoundInfo struct {
BoundType tree.WindowFrameBoundType
BoundType treewindow.WindowFrameBoundType
Bounds []windowFrameOffsetIsStartInfo
}

Expand Down Expand Up @@ -215,9 +215,9 @@ func getAssignFunc(typeFamily types.Family) assignFunc {
}

func getValueByOffsetOp(
bound tree.WindowFrameBoundType, isOrdColAsc bool,
bound treewindow.WindowFrameBoundType, isOrdColAsc bool,
) treebin.BinaryOperatorSymbol {
if bound == tree.OffsetFollowing {
if bound == treewindow.OffsetFollowing {
if isOrdColAsc {
return treebin.Plus
}
Expand Down
Loading

0 comments on commit 0caa03d

Please sign in to comment.