Skip to content

Commit

Permalink
exec: Add framework for vectorized casts.
Browse files Browse the repository at this point in the history
This PR adds in a framework for performing casts
within the vectorized engine, along with a few
casts implemented already. Additionally, it implements
a randomized testing framework for the cast operators.

Release note: None
  • Loading branch information
rohany committed Aug 29, 2019
1 parent 45c4543 commit a3ec07b
Show file tree
Hide file tree
Showing 9 changed files with 606 additions and 0 deletions.
2 changes: 2 additions & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -788,6 +788,7 @@ EXECGEN_TARGETS = \
pkg/col/coldata/vec.eg.go \
pkg/sql/exec/any_not_null_agg.eg.go \
pkg/sql/exec/avg_agg.eg.go \
pkg/sql/exec/cast.eg.go \
pkg/sql/exec/const.eg.go \
pkg/sql/exec/distinct.eg.go \
pkg/sql/exec/hashjoiner.eg.go \
Expand Down Expand Up @@ -1460,6 +1461,7 @@ $(SETTINGS_DOC_PAGE): $(settings-doc-gen)
pkg/col/coldata/vec.eg.go: pkg/col/coldata/vec_tmpl.go
pkg/sql/exec/any_not_null_agg.eg.go: pkg/sql/exec/any_not_null_agg_tmpl.go
pkg/sql/exec/avg_agg.eg.go: pkg/sql/exec/avg_agg_tmpl.go
pkg/sql/exec/cast.eg.go: pkg/sql/exec/cast_tmpl.go
pkg/sql/exec/const.eg.go: pkg/sql/exec/const_tmpl.go
pkg/sql/exec/distinct.eg.go: pkg/sql/exec/distinct_tmpl.go
pkg/sql/exec/hashjoiner.eg.go: pkg/sql/exec/hashjoiner_tmpl.go
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/distsqlrun/column_exec_setup.go
Original file line number Diff line number Diff line change
Expand Up @@ -768,6 +768,18 @@ func planProjectionOperators(
return planProjectionExpr(ctx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input)
case *tree.BinaryExpr:
return planProjectionExpr(ctx, t.Operator, t.ResolvedType(), t.TypedLeft(), t.TypedRight(), columnTypes, input)
case *tree.CastExpr:
op, resultIdx, ct, memUsed, err = planProjectionOperators(ctx, t.Expr.(tree.TypedExpr), columnTypes, input)
if err != nil {
return nil, 0, nil, 0, err
}
outputIdx := len(ct)
op, err = exec.GetCastOperator(op, resultIdx, outputIdx, t.Expr.(tree.TypedExpr).ResolvedType(), t.Type)
ct = append(ct, *t.Type)
if sMem, ok := op.(exec.StaticMemoryOperator); ok {
memUsed += sMem.EstimateStaticMemoryUsage()
}
return op, outputIdx, ct, memUsed, err
case *tree.FuncExpr:
var (
inputCols []int
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec/.gitignore
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
any_not_null_agg.eg.go
avg_agg.eg.go
cast.eg.go
const.eg.go
distinct.eg.go
hashjoiner.eg.go
Expand Down
104 changes: 104 additions & 0 deletions pkg/sql/exec/cast_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package exec

import (
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

func TestRandomizedCast(t *testing.T) {

datumAsBool := func(d tree.Datum) interface{} {
return bool(tree.MustBeDBool(d))
}
datumAsInt := func(d tree.Datum) interface{} {
return int(tree.MustBeDInt(d))
}
datumAsFloat := func(d tree.Datum) interface{} {
return float64(tree.MustBeDFloat(d))
}
datumAsDecimal := func(d tree.Datum) interface{} {
return tree.MustBeDDecimal(d).Decimal
}

tc := []struct {
fromTyp *types.T
fromPhysType func(tree.Datum) interface{}
toTyp *types.T
toPhysType func(tree.Datum) interface{}
// Some types casting can fail, so retry if we
// generate a datum that is unable to be casted.
retryGeneration bool
}{
//bool -> t tests
{types.Bool, datumAsBool, types.Bool, datumAsBool, false},
{types.Bool, datumAsBool, types.Int, datumAsInt, false},
{types.Bool, datumAsBool, types.Float, datumAsFloat, false},
// decimal -> t tests
{types.Decimal, datumAsDecimal, types.Bool, datumAsBool, false},
// int -> t tests
{types.Int, datumAsInt, types.Bool, datumAsBool, false},
{types.Int, datumAsInt, types.Float, datumAsFloat, false},
{types.Int, datumAsInt, types.Decimal, datumAsDecimal, false},
// float -> t tests
{types.Float, datumAsFloat, types.Bool, datumAsBool, false},
// We can sometimes generate a float outside of the range of the integers,
// so we want to retry with generation if that occurs.
{types.Float, datumAsFloat, types.Int, datumAsInt, true},
{types.Float, datumAsFloat, types.Decimal, datumAsDecimal, false},
}

evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings())
rng, _ := randutil.NewPseudoRand()

for _, c := range tc {
t.Run(fmt.Sprintf("%sTo%s", c.fromTyp.String(), c.toTyp.String()), func(t *testing.T) {
n := 100
// Make an input vector of length n.
input := tuples{}
output := tuples{}
for i := 0; i < n; i++ {
// We don't allow any NULL datums to be generated, so disable
// this ability in the RandDatum function.
fromDatum := sqlbase.RandDatum(rng, c.fromTyp, false)
var (
toDatum tree.Datum
err error
)
toDatum, err = tree.PerformCast(evalCtx, fromDatum, c.toTyp)
if c.retryGeneration {
for err != nil {
// If we are allowed to retry, make a new datum and cast it on error.
fromDatum = sqlbase.RandDatum(rng, c.fromTyp, false)
toDatum, err = tree.PerformCast(evalCtx, fromDatum, c.toTyp)
}
} else {
if err != nil {
t.Fatal(err)
}
}
input = append(input, tuple{c.fromPhysType(fromDatum)})
output = append(output, tuple{c.toPhysType(toDatum)})
}
runTests(t, []tuples{input}, output, orderedVerifier, []int{1},
func(input []Operator) (Operator, error) {
return GetCastOperator(input[0], 0 /* inputIdx*/, 1 /* resultIdx */, c.fromTyp, c.toTyp)
})
})
}
}
171 changes: 171 additions & 0 deletions pkg/sql/exec/cast_tmpl.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

// {{/*
// +build execgen_template
//
// This file is the execgen template for cast.eg.go. It's formatted in a
// special way, so it's both valid Go and a valid text/template input. This
// permits editing this file with editor support.
//
// */}}

package exec

import (
"context"
"math"

"github.com/cockroachdb/apd"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
"github.com/cockroachdb/cockroach/pkg/sql/exec/execerror"
"github.com/cockroachdb/cockroach/pkg/sql/exec/execgen"
"github.com/cockroachdb/cockroach/pkg/sql/exec/typeconv"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
semtypes "github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/pkg/errors"
)

// {{/*

type _ALLTYPES interface{}
type _OVERLOADTYPES interface{}
type _TOTYPE interface{}
type _GOTYPE interface{}
type _FROMTYPE interface{}

var _ apd.Decimal
var _ = math.MaxInt8
var _ tree.Datum

func _ASSIGN_CAST(to, from interface{}) {
execerror.VectorizedInternalPanic("")
}

// */}}

// Use execgen package to remove unused import warning.
var _ interface{} = execgen.GET

func GetCastOperator(
input Operator, colIdx int, resultIdx int, fromType *semtypes.T, toType *semtypes.T,
) (Operator, error) {
switch from := typeconv.FromColumnType(fromType); from {
// {{ range $typ, $overloads := . }}
case coltypes._ALLTYPES:
switch to := typeconv.FromColumnType(toType); to {
// {{ range $overloads }}
// {{ if isCastFuncSet . }}
case coltypes._OVERLOADTYPES:
return &castOp_FROMTYPE_TOTYPE{
OneInputNode: NewOneInputNode(input),
colIdx: colIdx,
outputIdx: resultIdx,
fromType: from,
toType: to,
}, nil
// {{end}}
// {{end}}
default:
return nil, errors.Errorf("unhandled cast FROM -> TO type: %s -> %s", from, to)
}
// {{end}}
default:
return nil, errors.Errorf("unhandled FROM type: %s", from)
}
}

// {{ range $typ, $overloads := . }}
// {{ range $overloads }}
// {{ if isCastFuncSet . }}

type castOp_FROMTYPE_TOTYPE struct {
OneInputNode
colIdx int
outputIdx int
fromType coltypes.T
toType coltypes.T
}

var _ StaticMemoryOperator = &castOp_FROMTYPE_TOTYPE{}

func (c *castOp_FROMTYPE_TOTYPE) EstimateStaticMemoryUsage() int {
return EstimateBatchSizeBytes([]coltypes.T{c.toType}, coldata.BatchSize)
}

func (c *castOp_FROMTYPE_TOTYPE) Init() {
c.input.Init()
}

func (c *castOp_FROMTYPE_TOTYPE) Next(ctx context.Context) coldata.Batch {
batch := c.input.Next(ctx)
n := batch.Length()
if n == 0 {
return batch
}
if c.outputIdx == batch.Width() {
batch.AppendCol(coltypes._TOTYPE)
}
vec := batch.ColVec(c.colIdx)
col := vec._FROMTYPE()
projVec := batch.ColVec(c.outputIdx)
projCol := projVec._TOTYPE()
if vec.MaybeHasNulls() {
vecNulls := vec.Nulls()
projNulls := projVec.Nulls()
if sel := batch.Selection(); sel != nil {
sel = sel[:n]
for _, i := range sel {
if vecNulls.NullAt(i) {
projNulls.SetNull(i)
} else {
v := execgen.GET(col, int(i))
var r _GOTYPE
_ASSIGN_CAST(r, v)
execgen.SET(projCol, int(i), r)
}
}
} else {
for execgen.RANGE(i, col) {
if vecNulls.NullAt(uint16(i)) {
projNulls.SetNull(uint16(i))
} else {
v := execgen.GET(col, i)
var r _GOTYPE
_ASSIGN_CAST(r, v)
execgen.SET(projCol, int(i), r)
}
}
}
} else {
if sel := batch.Selection(); sel != nil {
sel = sel[:n]
for _, i := range sel {
v := execgen.GET(col, int(i))
var r _GOTYPE
_ASSIGN_CAST(r, v)
execgen.SET(projCol, int(i), r)
}
} else {
for execgen.RANGE(i, col) {
v := execgen.GET(col, i)
var r _GOTYPE
_ASSIGN_CAST(r, v)
execgen.SET(projCol, int(i), r)
}
}
}
return batch
}

// {{end}}
// {{end}}
// {{end}}
52 changes: 52 additions & 0 deletions pkg/sql/exec/execgen/cmd/execgen/cast_gen.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package main

import (
"io"
"io/ioutil"
"strings"
"text/template"
)

func genCastOperators(wr io.Writer) error {
t, err := ioutil.ReadFile("pkg/sql/exec/cast_tmpl.go")
if err != nil {
return err
}

s := string(t)

assignCast := makeFunctionRegex("_ASSIGN_CAST", 2)
s = assignCast.ReplaceAllString(s, `{{.Assign "$1" "$2"}}`)
s = strings.Replace(s, "_ALLTYPES", "{{$typ}}", -1)
s = strings.Replace(s, "_OVERLOADTYPES", "{{.ToTyp}}", -1)
s = strings.Replace(s, "_FROMTYPE", "{{.FromTyp}}", -1)
s = strings.Replace(s, "_TOTYPE", "{{.ToTyp}}", -1)
s = strings.Replace(s, "_GOTYPE", "{{.ToGoTyp}}", -1)

s = replaceManipulationFuncs(".FromTyp", s)

isCastFuncSet := func(ov castOverload) bool {
return ov.AssignFunc != nil
}

tmpl, err := template.New("cast").Funcs(template.FuncMap{"isCastFuncSet": isCastFuncSet}).Parse(s)
if err != nil {
return err
}

return tmpl.Execute(wr, castOverloads)
}

func init() {
registerGenerator(genCastOperators, "cast.eg.go")
}
Loading

0 comments on commit a3ec07b

Please sign in to comment.