Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
47942: colexec: some optimizations r=yuzefovich a=yuzefovich

**colexec: remove one of the Go maps from hash aggregator**

This commit switches usage of `map` to iteration over `[]uint64` when
building selection vectors in the hash aggregator. This is a lot more
efficient when group sizes are relatively large with moderate hit when
group sizes are small. This hit is reduced in a follow-up commit.

Release note: None

**colexec: more improvements to hash aggregator**

This commit removes the buffering stage of the hash aggregator as well
as removes the "append only" scratch batch that we're currently using.
The removal of buffering stage allows us to have smaller buffers without
sacrificing the performance. The removal of the scratch batch allows to
avoid copying over the data from the input batch and using that input
batch directly. We will be descructively modifying the selection vector
on that batch, but such behavior is acceptable because hash aggregator
owns the output batch, and the input batch will not be propagated
further.

This commit also bumps `hashAggFuncsAllocSize` from 16 to 64 which
gives us minor performance improvement in case of small group sizes.

Release note: None

**colexec: remove some allocations**

In a recent PR (for logical types plumbing) I introduced some
unnecessary allocations for unhandled type case - by taking a pointer
from a value in `[]types.T` slice. This commit fixes that.

Release note: None

47953: colexec, coldata: fix compiler warnings in template files r=yuzefovich a=yuzefovich

This commit fixes all compiler warnings that I see in Goland. To get
there it does the following:
1. renames `Vec._TemplateType` to `Vec.TemplateType` so that the method
is considered exported
2. pulls out declaration of local variables outside of templated `if`
blocks
3. breaks up the chained function call to parse flags in `pkg/workload`
and a few other places so that there is an allocation of a struct and we
can call a method on it that has a pointer receiver. It shouldn't matter
for the performance though.

Release note: None

47974: roachtest: fail tpcdsvec test with an error r=yuzefovich a=yuzefovich

In `tpcdsvec` test we run all the queries even we hit an error.
Previously if an error occurred, we would just fail the test, and now we
will be failing with an error that is a "combination" of all occurred
errors.

Addresses: #47889.

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
craig[bot] and yuzefovich committed Apr 28, 2020
4 parents e770baa + 1444f90 + 0b2fd2b + c0a483d commit 211abed
Show file tree
Hide file tree
Showing 42 changed files with 238 additions and 229 deletions.
3 changes: 2 additions & 1 deletion pkg/ccl/importccl/read_import_workload.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,8 @@ func (w *workloadReader) readFiles(
}
gen := meta.New()
if f, ok := gen.(workload.Flagser); ok {
if err := f.Flags().Parse(conf.Flags); err != nil {
flags := f.Flags()
if err := flags.Parse(conf.Flags); err != nil {
return errors.Wrapf(err, `parsing parameters %s`, strings.Join(conf.Flags, ` `))
}
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/workloadccl/allccl/all_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,8 @@ func TestAllRegisteredSetup(t *testing.T) {
case `roachmart`:
// TODO(dan): It'd be nice to test this with the default flags. For now,
// this is better than nothing.
if err := gen.(workload.Flagser).Flags().Parse([]string{
flags := gen.(workload.Flagser).Flags()
if err := flags.Parse([]string{
`--users=10`, `--orders=100`, `--partition=false`,
}); err != nil {
t.Fatal(err)
Expand Down
9 changes: 5 additions & 4 deletions pkg/cmd/roachtest/tpcdsvec.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/cmpconn"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/workload/tpcds"
"github.com/cockroachdb/errors"
)

func registerTPCDSVec(r *testRegistry) {
Expand Down Expand Up @@ -149,7 +150,7 @@ func registerTPCDSVec(r *testRegistry) {
}

noStatsRunTimes := make(map[int]float64)
encounteredErrors := false
var errToReport error
// We will run all queries in two scenarios: without stats and with
// auto stats. The idea is that the plans are likely to be different,
// so we will be testing different execution scenarios. We additionally
Expand All @@ -175,7 +176,7 @@ func registerTPCDSVec(r *testRegistry) {
ctx, 3*timeout, conns, "", query, false, /* ignoreSQLErrors */
); err != nil {
t.Status(fmt.Sprintf("encountered an error: %s\n", err))
encounteredErrors = true
errToReport = errors.CombineErrors(errToReport, err)
} else {
runTimeInSeconds := timeutil.Since(start).Seconds()
t.Status(
Expand All @@ -198,8 +199,8 @@ func registerTPCDSVec(r *testRegistry) {
createStatsFromTables(t, clusterConn, tpcdsTables)
}
}
if encounteredErrors {
t.FailNow()
if errToReport != nil {
t.Fatal(errToReport)
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/col/coldata/unknown_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ func (u unknown) SetCol(interface{}) {
panic("Vec is of unknown type and should not be accessed")
}

func (u unknown) _TemplateType() []interface{} {
func (u unknown) TemplateType() []interface{} {
panic("Vec is of unknown type and should not be accessed")
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/col/coldata/vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ type Vec interface {

// TemplateType returns an []interface{} and is used for operator templates.
// Do not call this from normal code - it'll always panic.
_TemplateType() []interface{}
TemplateType() []interface{}

// Append uses SliceArgs to append elements of a source Vec into this Vec.
// It is logically equivalent to:
Expand Down Expand Up @@ -226,7 +226,7 @@ func (m *memColumn) Col() interface{} {
return m.col
}

func (m *memColumn) _TemplateType() []interface{} {
func (m *memColumn) TemplateType() []interface{} {
panic("don't call this from non template code")
}

Expand Down
14 changes: 7 additions & 7 deletions pkg/col/coldata/vec_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,8 @@ func (m *memColumn) Append(args SliceArgs) {
switch args.ColType {
// {{range .}}
case _TYPES_T:
fromCol := args.Src._TemplateType()
toCol := m._TemplateType()
fromCol := args.Src.TemplateType()
toCol := m.TemplateType()
// NOTE: it is unfortunate that we always append whole slice without paying
// attention to whether the values are NULL. However, if we do start paying
// attention, the performance suffers dramatically, so we choose to copy
Expand Down Expand Up @@ -153,8 +153,8 @@ func (m *memColumn) Copy(args CopySliceArgs) {
switch args.ColType {
// {{range .}}
case _TYPES_T:
fromCol := args.Src._TemplateType()
toCol := m._TemplateType()
fromCol := args.Src.TemplateType()
toCol := m.TemplateType()
if args.Sel != nil {
sel := args.Sel
if args.SelOnDest {
Expand All @@ -177,7 +177,7 @@ func (m *memColumn) Window(colType coltypes.T, start int, end int) Vec {
switch colType {
// {{range .}}
case _TYPES_T:
col := m._TemplateType()
col := m.TemplateType()
return &memColumn{
t: colType,
col: execgen.WINDOW(col, start, end),
Expand All @@ -195,7 +195,7 @@ func SetValueAt(v Vec, elem interface{}, rowIdx int, colType coltypes.T) {
switch colType {
// {{range .}}
case _TYPES_T:
target := v._TemplateType()
target := v.TemplateType()
newVal := elem.(_GOTYPE)
execgen.SET(target, rowIdx, newVal)
// {{end}}
Expand All @@ -210,7 +210,7 @@ func GetValueAt(v Vec, rowIdx int, colType coltypes.T) interface{} {
switch colType {
// {{range .}}
case _TYPES_T:
target := v._TemplateType()
target := v.TemplateType()
return execgen.UNSAFEGET(target, rowIdx)
// {{end}}
default:
Expand Down
20 changes: 11 additions & 9 deletions pkg/sql/colexec/and_or_projection_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,10 +131,12 @@ func (o *_OP_LOWERProjOp) Next(ctx context.Context) coldata.Batch {
//
// knownResult indicates the boolean value which if present on the left side
// fully determines the result of the logical operation.
var (
knownResult bool
isLeftNull, isRightNull bool
)
// {{ if _IS_OR_OP }}
knownResult := true
// {{ else }}
knownResult := false
knownResult = true
// {{ end }}
leftCol := batch.ColVec(o.leftIdx)
leftColVals := leftCol.Bool()
Expand Down Expand Up @@ -234,9 +236,9 @@ func (o *_OP_LOWERProjOp) Next(ctx context.Context) coldata.Batch {
func _ADD_TUPLE_FOR_RIGHT(_L_HAS_NULLS bool) { // */}}
// {{define "addTupleForRight" -}}
// {{if _L_HAS_NULLS}}
isLeftNull := leftNulls.NullAt(i)
isLeftNull = leftNulls.NullAt(i)
// {{else}}
isLeftNull := false
isLeftNull = false
// {{end}}
if isLeftNull || leftColVals[i] != knownResult {
// We add the tuple into the selection vector if the left value is NULL or
Expand Down Expand Up @@ -280,18 +282,18 @@ func _SET_VALUES(_IS_OR_OP bool, _L_HAS_NULLS bool, _R_HAS_NULLS bool) { // */}}
func _SET_SINGLE_VALUE(_IS_OR_OP bool, _L_HAS_NULLS bool, _R_HAS_NULLS bool) { // */}}
// {{ define "setSingleValue" -}}
// {{ if _L_HAS_NULLS }}
isLeftNull := leftNulls.NullAt(idx)
isLeftNull = leftNulls.NullAt(idx)
// {{ else }}
isLeftNull := false
isLeftNull = false
// {{ end }}
leftVal := leftColVals[idx]
if !isLeftNull && leftVal == knownResult {
outputColVals[idx] = leftVal
} else {
// {{ if _R_HAS_NULLS }}
isRightNull := rightNulls.NullAt(idx)
isRightNull = rightNulls.NullAt(idx)
// {{ else }}
isRightNull := false
isRightNull = false
// {{ end }}
rightVal := rightColVals[idx]
// {{ if _IS_OR_OP }}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/colexec/any_not_null_agg_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ type anyNotNull_TYPEAgg struct {
func (a *anyNotNull_TYPEAgg) Init(groups []bool, vec coldata.Vec) {
a.groups = groups
a.vec = vec
a.col = vec._TemplateType()
a.col = vec.TemplateType()
a.nulls = vec.Nulls()
a.Reset()
}
Expand Down Expand Up @@ -131,7 +131,7 @@ func (a *anyNotNull_TYPEAgg) Compute(b coldata.Batch, inputIdxs []uint32) {
return
}
vec, sel := b.ColVec(int(inputIdxs[0])), b.Selection()
col, nulls := vec._TemplateType(), vec.Nulls()
col, nulls := vec.TemplateType(), vec.Nulls()

a.allocator.PerformOperation(
[]coldata.Vec{a.vec},
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/colexec/avg_agg_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ var _ aggregateFunc = &avg_TYPEAgg{}

func (a *avg_TYPEAgg) Init(groups []bool, v coldata.Vec) {
a.groups = groups
a.scratch.vec = v._TemplateType()
a.scratch.vec = v.TemplateType()
a.scratch.nulls = v.Nulls()
a.Reset()
}
Expand Down Expand Up @@ -141,7 +141,7 @@ func (a *avg_TYPEAgg) Compute(b coldata.Batch, inputIdxs []uint32) {
return
}
vec, sel := b.ColVec(int(inputIdxs[0])), b.Selection()
col, nulls := vec._TemplateType(), vec.Nulls()
col, nulls := vec.TemplateType(), vec.Nulls()
if nulls.MaybeHasNulls() {
if sel != nil {
sel = sel[:inputLen]
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/const_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ func (c const_TYPEOp) Next(ctx context.Context) coldata.Batch {
return coldata.ZeroBatch
}
vec := batch.ColVec(c.outputIdx)
col := vec._TemplateType()
col := vec.TemplateType()
if vec.MaybeHasNulls() {
// We need to make sure that there are no left over null values in the
// output vector.
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/colexec/distinct_tmpl.go
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,7 @@ func (p *sortedDistinct_TYPEOp) Next(ctx context.Context) coldata.Batch {
if vec.MaybeHasNulls() {
nulls = vec.Nulls()
}
col := vec._TemplateType()
col := vec.TemplateType()

// We always output the first row.
lastVal := p.lastVal
Expand Down Expand Up @@ -311,7 +311,7 @@ func (p partitioner_TYPE) partitionWithOrder(
nulls = colVec.Nulls()
}

col := colVec._TemplateType()
col := colVec.TemplateType()
col = execgen.SLICE(col, 0, n)
outputCol = outputCol[:n]
outputCol[0] = true
Expand All @@ -336,7 +336,7 @@ func (p partitioner_TYPE) partition(colVec coldata.Vec, outputCol []bool, n int)
nulls = colVec.Nulls()
}

col := colVec._TemplateType()
col := colVec.TemplateType()
col = execgen.SLICE(col, 0, n)
outputCol = outputCol[:n]
outputCol[0] = true
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func genAnyNotNullAgg(wr io.Writer) error {
s = strings.Replace(s, "_GOTYPE", "{{.LTyp.GoTypeName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = strings.Replace(s, "_TYPE", "{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)

findAnyNotNull := makeFunctionRegex("_FIND_ANY_NOT_NULL", 4)
s = findAnyNotNull.ReplaceAllString(s, `{{template "findAnyNotNull" buildDict "Global" . "LTyp" .LTyp "HasNulls" $4}}`)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ func genAvgAgg(wr io.Writer) error {
s = strings.Replace(s, "_GOTYPE", "{{.Type.GoTypeName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.Type}}", -1)
s = strings.Replace(s, "_TYPE", "{{.Type}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.Type}}", -1)
s = strings.Replace(s, "TemplateType", "{{.Type}}", -1)

assignDivRe := makeFunctionRegex("_ASSIGN_DIV_INT64", 3)
s = assignDivRe.ReplaceAllString(s, makeTemplateFunctionCall("AssignDivInt64", 3))
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/const_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func genConstOps(wr io.Writer) error {
s = strings.Replace(s, "_GOTYPE", "{{.GoTypeName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.}}", -1)
s = strings.Replace(s, "_TYPE", "{{.}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.}}", -1)
s = strings.Replace(s, "TemplateType", "{{.}}", -1)
s = replaceManipulationFuncs("", s)

// Now, generate the op, from the template.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ func genDistinctOps(wr io.Writer) error {
s = strings.Replace(s, "_GOTYPESLICE", "{{.LTyp.GoTypeSliceName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = strings.Replace(s, "_TYPE", "{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)

assignNeRe := makeFunctionRegex("_ASSIGN_NE", 3)
s = assignNeRe.ReplaceAllString(s, makeTemplateFunctionCall("Global.Assign", 3))
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/hash_aggregator_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,13 +29,16 @@ func genHashAggregator(wr io.Writer) error {

s := string(t)

s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = replaceManipulationFuncs(".Global.LTyp", s)

assignCmpRe := makeFunctionRegex("_ASSIGN_NE", 3)
s = assignCmpRe.ReplaceAllString(s, makeTemplateFunctionCall("Global.Assign", 3))

populateSels := makeFunctionRegex("_POPULATE_SELS", 3)
s = populateSels.ReplaceAllString(s, `{{template "populateSels" buildDict "Global" . "BatchHasSelection" $3}}`)

matchLoop := makeFunctionRegex("_MATCH_LOOP", 8)
s = matchLoop.ReplaceAllString(
s, `{{template "matchLoop" buildDict "Global" . "LhsMaybeHasNulls" $7 "RhsMaybeHasNulls" $8}}`)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/hash_utils_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ func genHashUtils(wr io.Writer) error {

s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = strings.Replace(s, "_TYPE", "{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)

assignHash := makeFunctionRegex("_ASSIGN_HASH", 2)
s = assignHash.ReplaceAllString(s, makeTemplateFunctionCall("Global.UnaryAssign", 2))
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/mergejoinbase_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func genMergeJoinBase(wr io.Writer) error {
// Replace the template variables.
s = strings.Replace(s, "_GOTYPE", "{{.LTyp.GoTypeName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)

assignEqRe := makeFunctionRegex("_ASSIGN_EQ", 3)
s = assignEqRe.ReplaceAllString(s, makeTemplateFunctionCall("Assign", 3))
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/mergejoiner_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ func genMergeJoinOps(wr io.Writer, jti joinTypeInfo) error {
s = strings.Replace(s, "_GOTYPESLICE", "{{.LTyp.GoTypeSliceName}}", -1)
s = strings.Replace(s, "_GOTYPE", "{{.LTyp.GoTypeName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "_L_SEL_IND", "{{$sel.LSelString}}", -1)
s = strings.Replace(s, "_R_SEL_IND", "{{$sel.RSelString}}", -1)
s = strings.Replace(s, "_IS_L_SEL", "{{$sel.IsLSel}}", -1)
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/colexec/execgen/cmd/execgen/projection_ops_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,11 +79,13 @@ func replaceProjConstTmplVariables(tmpl string, isConstLeft bool) string {
tmpl = strings.Replace(tmpl, "_CONST_SIDE", "L", -1)
tmpl = strings.Replace(tmpl, "_IS_CONST_LEFT", "true", -1)
tmpl = strings.Replace(tmpl, "_OP_CONST_NAME", "proj{{.Name}}{{.LTyp}}Const{{.RTyp}}Op", -1)
tmpl = strings.Replace(tmpl, "_NON_CONST_GOTYPESLICE", "{{.RTyp.GoTypeSliceName}}", -1)
tmpl = replaceManipulationFuncs(".RTyp", tmpl)
} else {
tmpl = strings.Replace(tmpl, "_CONST_SIDE", "R", -1)
tmpl = strings.Replace(tmpl, "_IS_CONST_LEFT", "false", -1)
tmpl = strings.Replace(tmpl, "_OP_CONST_NAME", "proj{{.Name}}{{.LTyp}}{{.RTyp}}ConstOp", -1)
tmpl = strings.Replace(tmpl, "_NON_CONST_GOTYPESLICE", "{{.LTyp.GoTypeSliceName}}", -1)
tmpl = replaceManipulationFuncs(".LTyp", tmpl)
}
return replaceProjTmplVariables(tmpl)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/rowstovec_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func genRowsToVec(wr io.Writer) error {
s := string(f)

// Replace the template variables.
s = strings.Replace(s, "_TemplateType", "{{.ExecType.String}}", -1)
s = strings.Replace(s, "TemplateType", "{{.ExecType.String}}", -1)
s = strings.Replace(s, "_GOTYPE", "{{.ExecType.GoTypeName}}", -1)
s = strings.Replace(s, "_FAMILY", "types.{{.Family}}", -1)
s = strings.Replace(s, "_WIDTH", "{{.Width}}", -1)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/select_in_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func genSelectIn(wr io.Writer) error {
s = assignEq.ReplaceAllString(s, makeTemplateFunctionCall("Assign", 3))
s = strings.Replace(s, "_GOTYPE", "{{.LGoType}}", -1)
s = strings.Replace(s, "_TYPE", "{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)

s = replaceManipulationFuncs(".LTyp", s)

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/sort_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func genSortOps(wr io.Writer) error {
s = strings.Replace(s, "_TYPE", "{{$typ}}", -1)
s = strings.Replace(s, "_DIR_ENUM", "{{.Dir}}", -1)
s = strings.Replace(s, "_DIR", "{{.DirString}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "_ISNULL", "{{$isNull}}", -1)
s = strings.Replace(s, "_HANDLES_NULLS", "{{if .Nulls}}WithNulls{{else}}{{end}}", -1)

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func genSumAgg(wr io.Writer) error {
s = strings.Replace(s, "_GOTYPE", "{{.LTyp.GoTypeName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = strings.Replace(s, "_TYPE", "{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)

assignAddRe := makeFunctionRegex("_ASSIGN_ADD", 3)
s = assignAddRe.ReplaceAllString(s, makeTemplateFunctionCall("Global.Assign", 3))
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/execgen/cmd/execgen/values_differ_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func genValuesDiffer(wr io.Writer) error {
s = strings.Replace(s, "_GOTYPE", "{{.LTyp.GoTypeName}}", -1)
s = strings.Replace(s, "_TYPES_T", "coltypes.{{.LTyp}}", -1)
s = strings.Replace(s, "_TYPE", "{{.LTyp}}", -1)
s = strings.Replace(s, "_TemplateType", "{{.LTyp}}", -1)
s = strings.Replace(s, "TemplateType", "{{.LTyp}}", -1)

assignNeRe := makeFunctionRegex("_ASSIGN_NE", 3)
s = assignNeRe.ReplaceAllString(s, makeTemplateFunctionCall("Assign", 3))
Expand Down
Loading

0 comments on commit 211abed

Please sign in to comment.