diff --git a/Makefile b/Makefile index f0fc5b52ee7c..988459a94861 100644 --- a/Makefile +++ b/Makefile @@ -784,6 +784,7 @@ DOCGEN_TARGETS := bin/.docgen_bnfs bin/.docgen_functions EXECGEN_TARGETS = \ pkg/sql/exec/any_not_null_agg.eg.go \ pkg/sql/exec/avg_agg.eg.go \ + pkg/sql/exec/cast_operator.eg.go \ pkg/sql/exec/coldata/vec.eg.go \ pkg/sql/exec/const.eg.go \ pkg/sql/exec/distinct.eg.go \ @@ -1459,6 +1460,7 @@ $(SETTINGS_DOC_PAGE): $(settings-doc-gen) 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_operator.eg.go: pkg/sql/exec/cast_operator_tmpl.go pkg/sql/exec/coldata/vec.eg.go: pkg/sql/exec/coldata/vec_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 diff --git a/pkg/sql/distsqlrun/column_exec_setup.go b/pkg/sql/distsqlrun/column_exec_setup.go index 322d947412cf..099dd28c39bc 100644 --- a/pkg/sql/distsqlrun/column_exec_setup.go +++ b/pkg/sql/distsqlrun/column_exec_setup.go @@ -766,6 +766,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) + outputIdx := len(ct) + op, err = exec.GetCastOperator(op, resultIdx, outputIdx, t.Expr.(tree.TypedExpr).ResolvedType(), t.Type) + if err != nil { + return nil, 0, nil, 0, err + } + 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 diff --git a/pkg/sql/exec/.gitignore b/pkg/sql/exec/.gitignore index dd6aaa2cefc6..803d7194db3c 100644 --- a/pkg/sql/exec/.gitignore +++ b/pkg/sql/exec/.gitignore @@ -1,5 +1,6 @@ any_not_null_agg.eg.go avg_agg.eg.go +cast_operator.eg.go coldata/vec.eg.go const.eg.go distinct.eg.go diff --git a/pkg/sql/exec/cast_operator_tmpl.go b/pkg/sql/exec/cast_operator_tmpl.go new file mode 100644 index 000000000000..64bcbf55dd3a --- /dev/null +++ b/pkg/sql/exec/cast_operator_tmpl.go @@ -0,0 +1,153 @@ +// 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_operator.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" + + "github.com/cockroachdb/apd" + "github.com/cockroachdb/cockroach/pkg/sql/exec/types/conv" + "github.com/pkg/errors" + + "github.com/cockroachdb/cockroach/pkg/sql/exec/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/exec/types" + semtypes "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +// {{/* + +type _ALLTYPES interface{} +type _OVERLOADTYPES interface{} +type _TOTYPE interface{} +type _FROMTYPE interface{} + +var _ apd.Decimal + +//var _ types.T +//var _ bytes.Buffer + +func _ASSIGN_CAST(to, from interface{}) { + panic("") +} + +// */}} + +func GetCastOperator( + input Operator, colIdx int, resultIdx int, fromType *semtypes.T, toType *semtypes.T, +) (Operator, error) { + switch from := conv.FromColumnType(fromType); from { + // {{ range $typ, $overloads := . }} + case types._ALLTYPES: + switch to := conv.FromColumnType(toType); to { + // {{ range $overloads }} + // {{ if isCastFuncSet . }} + case types._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 types.T + toType types.T +} + +var _ StaticMemoryOperator = &castOp_FROMTYPE_TOTYPE{} + +func (c *castOp_FROMTYPE_TOTYPE) EstimateStaticMemoryUsage() int { + return EstimateBatchSizeBytes([]types.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(types._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 { + for _, i := range sel { + if vecNulls.NullAt(uint16(i)) { + projNulls.SetNull(uint16(i)) + } else { + _ASSIGN_CAST(projCol[i], col[i]) + } + } + } else { + col = col[:n] + projCol = projCol[:n] + for i := range col { + if vecNulls.NullAt(uint16(i)) { + projNulls.SetNull(uint16(i)) + } else { + _ASSIGN_CAST(projCol[i], col[i]) + } + } + } + } else { + if sel := batch.Selection(); sel != nil { + for _, i := range sel { + _ASSIGN_CAST(projCol[i], col[i]) + } + } else { + for i := range col { + _ASSIGN_CAST(projCol[i], col[i]) + } + } + } + return batch +} + +// {{end}} +// {{end}} +// {{end}} diff --git a/pkg/sql/exec/execgen/cmd/execgen/cast_operator_gen.go b/pkg/sql/exec/execgen/cmd/execgen/cast_operator_gen.go new file mode 100644 index 000000000000..d2d82955ebb1 --- /dev/null +++ b/pkg/sql/exec/execgen/cmd/execgen/cast_operator_gen.go @@ -0,0 +1,49 @@ +// 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_operator_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) + + isCastFuncSet := func(ov castOverload) bool { + return ov.AssignFunc != nil + } + + tmpl, err := template.New("cast_operator").Funcs(template.FuncMap{"isCastFuncSet": isCastFuncSet}).Parse(s) + if err != nil { + return err + } + + return tmpl.Execute(wr, castOverloads) +} + +func init() { + registerGenerator(genCastOperators, "cast_operator.eg.go") +} diff --git a/pkg/sql/exec/execgen/cmd/execgen/overloads.go b/pkg/sql/exec/execgen/cmd/execgen/overloads.go index 4345ebdf09f1..3a826e74f51b 100644 --- a/pkg/sql/exec/execgen/cmd/execgen/overloads.go +++ b/pkg/sql/exec/execgen/cmd/execgen/overloads.go @@ -141,6 +141,31 @@ func (o overload) UnaryAssign(target, v string) string { return fmt.Sprintf("%s = %s(%s)", target, o.OpStr, v) } +type castOverload struct { + FromTyp types.T + ToTyp types.T + AssignFunc castAssignFunc +} + +func (o castOverload) Assign(to, from string) string { + return o.AssignFunc(to, from) +} + +type castAssignFunc func(to, from string) string + +func castIdentity(to, from string) string { + return fmt.Sprintf("%s = %s", to, from) +} + +func intToDecimal(to, from string) string { + convStr := ` + %[1]s = *apd.New(int64(%[2]s), 0) + ` + return fmt.Sprintf(convStr, to, from) +} + +var castOverloads map[types.T][]castOverload + func init() { registerTypeCustomizers() @@ -218,6 +243,189 @@ func init() { } hashOverloads = append(hashOverloads, ov) } + + // Build cast overloads. + castOverloads = make(map[types.T][]castOverload) + _ = castOverloads + for _, from := range inputTypes { + switch from { + case types.Bool: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + ov.AssignFunc = castIdentity + case types.Bytes: + // TODO (rohany): Implement bool -> string casting. + continue + case types.Decimal: + // TODO (rohany): Implement bool -> decimal casting. + continue + case types.Int8: + fallthrough + case types.Int16: + fallthrough + case types.Int32: + fallthrough + case types.Int64: + fallthrough + case types.Float32: + fallthrough + case types.Float64: + ov.AssignFunc = func(to, from string) string { + convStr := ` + %[1]s = 0 + if %[2]s { + %[1]s = 1 + } + ` + return fmt.Sprintf(convStr, to, from) + } + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Bytes: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + ov.AssignFunc = castIdentity + case types.Decimal: + case types.Int8: + case types.Int16: + case types.Int32: + case types.Int64: + case types.Float32: + case types.Float64: + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Decimal: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + case types.Decimal: + ov.AssignFunc = castIdentity + case types.Int8: + case types.Int16: + case types.Int32: + case types.Int64: + case types.Float32: + case types.Float64: + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Int8: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + case types.Decimal: + ov.AssignFunc = intToDecimal + case types.Int8: + ov.AssignFunc = castIdentity + case types.Int16: + case types.Int32: + case types.Int64: + case types.Float32: + case types.Float64: + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Int16: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + case types.Decimal: + ov.AssignFunc = intToDecimal + case types.Int8: + case types.Int16: + ov.AssignFunc = castIdentity + case types.Int32: + case types.Int64: + case types.Float32: + case types.Float64: + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Int32: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + case types.Decimal: + ov.AssignFunc = intToDecimal + case types.Int8: + case types.Int16: + case types.Int32: + ov.AssignFunc = castIdentity + case types.Int64: + case types.Float32: + case types.Float64: + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Int64: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + case types.Decimal: + ov.AssignFunc = intToDecimal + case types.Int8: + case types.Int16: + case types.Int32: + case types.Int64: + ov.AssignFunc = castIdentity + case types.Float32: + case types.Float64: + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Float32: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + case types.Decimal: + case types.Int8: + case types.Int16: + case types.Int32: + case types.Int64: + case types.Float32: + ov.AssignFunc = castIdentity + case types.Float64: + } + castOverloads[from] = append(castOverloads[from], ov) + } + case types.Float64: + for _, to := range inputTypes { + ov := castOverload{FromTyp: from, ToTyp: to} + switch to { + case types.Bool: + case types.Bytes: + case types.Decimal: + case types.Int8: + case types.Int16: + case types.Int32: + case types.Int64: + case types.Float32: + case types.Float64: + ov.AssignFunc = castIdentity + } + castOverloads[from] = append(castOverloads[from], ov) + } + } + } } // typeCustomizer is a marker interface for something that implements one or diff --git a/pkg/sql/exec/select_in_tmpl.go b/pkg/sql/exec/select_in_tmpl.go index 82574eab1c63..888553f22212 100644 --- a/pkg/sql/exec/select_in_tmpl.go +++ b/pkg/sql/exec/select_in_tmpl.go @@ -11,7 +11,7 @@ // {{/* // +build execgen_template // -// This file is the execgen template for distinct.eg.go. It's formatted in a +// This file is the execgen template for select_in.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. //