From c106c32b8106e2be42193ac6e329f5bc9979e805 Mon Sep 17 00:00:00 2001 From: Solon Gordon Date: Thu, 11 Oct 2018 17:13:11 -0400 Subject: [PATCH] exec: initial commit of execgen tool Execgen will be our tool for generating templated code necessary for columnarized execution. So far it only generates the EncDatumRowsToColVec function, which is used by the columnarizer to convert a RowSource into a columnarized Operator. Release note: None --- Makefile | 16 +- build/variables.mk | 2 + pkg/sql/distsqlrun/columnarizer.go | 116 +----------- pkg/sql/exec/.gitignore | 1 + pkg/sql/exec/execgen/cmd/execgen/main.go | 160 +++++++++++++++++ .../exec/execgen/cmd/execgen/rowstovec_gen.go | 165 ++++++++++++++++++ pkg/sql/exec/rowstovec_test.go | 99 +++++++++++ pkg/sql/exec/types/t_string.go | 16 ++ pkg/sql/exec/types/types.go | 1 + 9 files changed, 459 insertions(+), 117 deletions(-) create mode 100644 pkg/sql/exec/.gitignore create mode 100644 pkg/sql/exec/execgen/cmd/execgen/main.go create mode 100644 pkg/sql/exec/execgen/cmd/execgen/rowstovec_gen.go create mode 100644 pkg/sql/exec/rowstovec_test.go create mode 100644 pkg/sql/exec/types/t_string.go diff --git a/Makefile b/Makefile index dc84eb166054..e53f0be2a4bd 100644 --- a/Makefile +++ b/Makefile @@ -694,6 +694,8 @@ PROTOBUF_TARGETS := bin/.go_protobuf_sources bin/.gw_protobuf_sources bin/.cpp_p DOCGEN_TARGETS := bin/.docgen_bnfs bin/.docgen_functions +EXECGEN_TARGETS = pkg/sql/exec/rowstovec.og.go + OPTGEN_TARGETS = \ pkg/sql/opt/memo/expr.og.go \ pkg/sql/opt/operator.og.go \ @@ -735,7 +737,7 @@ BUILDINFO = .buildinfo/tag .buildinfo/rev BUILD_TAGGED_RELEASE = $(go-targets): bin/.bootstrap $(BUILDINFO) $(CGO_FLAGS_FILES) $(PROTOBUF_TARGETS) -$(go-targets): $(SQLPARSER_TARGETS) $(OPTGEN_TARGETS) +$(go-targets): $(SQLPARSER_TARGETS) $(EXECGEN_TARGETS) $(OPTGEN_TARGETS) $(go-targets): override LINKFLAGS += \ -X "github.com/cockroachdb/cockroach/pkg/build.tag=$(shell cat .buildinfo/tag)" \ -X "github.com/cockroachdb/cockroach/pkg/build.rev=$(shell cat .buildinfo/rev)" \ @@ -914,7 +916,7 @@ dupl: bin/.bootstrap .PHONY: generate generate: ## Regenerate generated code. -generate: protobuf $(DOCGEN_TARGETS) $(OPTGEN_TARGETS) $(SQLPARSER_TARGETS) $(SETTINGS_DOC_PAGE) bin/langgen +generate: protobuf $(DOCGEN_TARGETS) $(EXECGEN_TARGETS) $(OPTGEN_TARGETS) $(SQLPARSER_TARGETS) $(SETTINGS_DOC_PAGE) bin/langgen $(GO) generate $(GOFLAGS) -tags '$(TAGS)' -ldflags '$(LINKFLAGS)' $(PKG) .PHONY: lint @@ -962,6 +964,7 @@ $(ARCHIVE): $(ARCHIVE).tmp ARCHIVE_EXTRAS = \ $(BUILDINFO) \ $(SQLPARSER_TARGETS) \ + $(EXECGEN_TARGETS) \ $(OPTGEN_TARGETS) \ pkg/ui/distccl/bindata.go pkg/ui/distoss/bindata.go @@ -1305,6 +1308,9 @@ settings-doc-gen := $(if $(filter buildshort,$(MAKECMDGOALS)),$(COCKROACHSHORT), $(SETTINGS_DOC_PAGE): $(settings-doc-gen) @$(settings-doc-gen) gen settings-list --format=html > $@ +pkg/sql/exec/%.og.go: bin/execgen + execgen $@ + optgen-defs := pkg/sql/opt/ops/*.opt optgen-norm-rules := pkg/sql/opt/norm/rules/*.opt optgen-xform-rules := pkg/sql/opt/xform/rules/*.opt @@ -1362,7 +1368,7 @@ clean: clean-c-deps .PHONY: maintainer-clean maintainer-clean: ## Like clean, but also remove some auto-generated source code. maintainer-clean: clean ui-maintainer-clean - rm -f $(SQLPARSER_TARGETS) $(OPTGEN_TARGETS) $(UI_PROTOS_OSS) $(UI_PROTOS_CCL) + rm -f $(SQLPARSER_TARGETS) $(EXECGEN_TARGETS) $(OPTGEN_TARGETS) $(UI_PROTOS_OSS) $(UI_PROTOS_CCL) .PHONY: unsafe-clean unsafe-clean: ## Like maintainer-clean, but also remove ALL untracked/ignored files. @@ -1380,6 +1386,7 @@ bins = \ bin/cockroach-oss \ bin/cockroach-short \ bin/docgen \ + bin/execgen \ bin/generate-binary \ bin/github-post \ bin/github-pull-request-make \ @@ -1403,6 +1410,7 @@ testbins = \ bin/logictestccl # Mappings for binaries that don't live in pkg/cmd. +execgen-package = ./pkg/sql/exec/execgen/cmd/execgen langgen-package = ./pkg/sql/opt/optgen/cmd/langgen optgen-package = ./pkg/sql/opt/optgen/cmd/optgen logictest-package = ./pkg/sql/logictest @@ -1414,7 +1422,7 @@ logictest-bins := bin/logictest bin/logictestopt bin/logictestccl # Additional dependencies for binaries that depend on generated code. # # TODO(benesch): Derive this automatically. This is getting out of hand. -bin/workload bin/docgen bin/roachtest $(logictest-bins): $(SQLPARSER_TARGETS) $(PROTOBUF_TARGETS) +bin/workload bin/docgen bin/execgen bin/roachtest $(logictest-bins): $(SQLPARSER_TARGETS) $(PROTOBUF_TARGETS) bin/workload bin/roachtest $(logictest-bins): $(C_LIBS_CCL) $(CGO_FLAGS_FILES) bin/roachtest bin/logictestopt: $(OPTGEN_TARGETS) diff --git a/build/variables.mk b/build/variables.mk index 5fabcac993e0..e3025acefd62 100644 --- a/build/variables.mk +++ b/build/variables.mk @@ -44,6 +44,7 @@ define VALID_VARS C_LIBS_OSS DOCGEN_TARGETS DUPLFLAGS + EXECGEN_TARGETS EXTRA_XCMAKE_FLAGS EXTRA_XCONFIGURE_FLAGS FILES @@ -156,6 +157,7 @@ define VALID_VARS cmake-flags configure-flags cyan + execgen-package go-targets go-targets-ccl have-defs diff --git a/pkg/sql/distsqlrun/columnarizer.go b/pkg/sql/distsqlrun/columnarizer.go index f8184d2101f5..d5f5067ad5e1 100644 --- a/pkg/sql/distsqlrun/columnarizer.go +++ b/pkg/sql/distsqlrun/columnarizer.go @@ -15,13 +15,9 @@ package distsqlrun import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/sql/exec" "github.com/cockroachdb/cockroach/pkg/sql/exec/types" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" - "github.com/cockroachdb/cockroach/pkg/util/encoding" ) // columnarizer turns a RowSource input into an exec.Operator output, by reading @@ -91,115 +87,9 @@ func (c *columnarizer) Next() exec.ColBatch { // Write each column into the output batch. for idx, ct := range columnTypes { - vec := c.batch.ColVec(idx) - switch ct.SemanticType { - // TODO(solon): these should be autogenerated from a template. - case sqlbase.ColumnType_BOOL: - col := vec.Bool() - for i := uint16(0); i < nRows; i++ { - ed := c.buffered[i][idx] - if err := ed.EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - if ed.Datum == tree.DNull { - vec.SetNull(i) - } - col.Set(i, bool(*ed.Datum.(*tree.DBool))) - } - case sqlbase.ColumnType_INT: - switch ct.Width { - case 8: - col := vec.Int8() - for i := uint16(0); i < nRows; i++ { - ed := c.buffered[i][idx] - if err := ed.EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - if ed.Datum == tree.DNull { - vec.SetNull(i) - } - col[i] = int8(*ed.Datum.(*tree.DInt)) - } - case 16: - col := vec.Int16() - for i := uint16(0); i < nRows; i++ { - ed := c.buffered[i][idx] - if err := ed.EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - if ed.Datum == tree.DNull { - vec.SetNull(i) - } - col[i] = int16(*ed.Datum.(*tree.DInt)) - } - case 32: - col := vec.Int32() - for i := uint16(0); i < nRows; i++ { - ed := c.buffered[i][idx] - if err := ed.EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - if ed.Datum == tree.DNull { - vec.SetNull(i) - } - col[i] = int32(*ed.Datum.(*tree.DInt)) - } - case 0, 64: - col := vec.Int64() - for i := uint16(0); i < nRows; i++ { - if c.buffered[i][idx].Datum == nil { - if err := c.buffered[i][idx].EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - } - if c.buffered[i][idx].Datum == tree.DNull { - vec.SetNull(i) - } - col[i] = int64(*c.buffered[i][idx].Datum.(*tree.DInt)) - } - default: - panic(fmt.Sprintf("integer with unknown width %d", ct.Width)) - } - case sqlbase.ColumnType_FLOAT: - col := vec.Float64() - for i := uint16(0); i < nRows; i++ { - ed := c.buffered[i][idx] - if err := ed.EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - if ed.Datum == tree.DNull { - vec.SetNull(i) - } - col[i] = float64(*ed.Datum.(*tree.DFloat)) - } - case sqlbase.ColumnType_BYTES: - col := vec.Bytes() - for i := uint16(0); i < nRows; i++ { - if c.buffered[i][idx].Datum == nil { - if err := c.buffered[i][idx].EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - } - if c.buffered[i][idx].Datum == tree.DNull { - vec.SetNull(i) - } - col.Set(i, encoding.UnsafeConvertStringToBytes(string(*c.buffered[i][idx].Datum.(*tree.DBytes)))) - } - case sqlbase.ColumnType_STRING: - col := vec.Bytes() - for i := uint16(0); i < nRows; i++ { - if c.buffered[i][idx].Datum == nil { - if err := c.buffered[i][idx].EnsureDecoded(&ct, &c.da); err != nil { - panic(err) - } - } - if c.buffered[i][idx].Datum == tree.DNull { - vec.SetNull(i) - } - col.Set(i, encoding.UnsafeConvertStringToBytes(string(*c.buffered[i][idx].Datum.(*tree.DString)))) - } - default: - panic(fmt.Sprintf("Unsupported column type %s", ct.SQLString())) + err := exec.EncDatumRowsToColVec(c.buffered[:nRows], c.batch.ColVec(idx), idx, &ct, &c.da) + if err != nil { + panic(err) } } return c.batch diff --git a/pkg/sql/exec/.gitignore b/pkg/sql/exec/.gitignore new file mode 100644 index 000000000000..e9a47673061b --- /dev/null +++ b/pkg/sql/exec/.gitignore @@ -0,0 +1 @@ +*.og.go diff --git a/pkg/sql/exec/execgen/cmd/execgen/main.go b/pkg/sql/exec/execgen/cmd/execgen/main.go new file mode 100644 index 000000000000..ed11a8d3979c --- /dev/null +++ b/pkg/sql/exec/execgen/cmd/execgen/main.go @@ -0,0 +1,160 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package main + +import ( + "bytes" + "flag" + "fmt" + "go/format" + "io" + "os" + "path/filepath" + + "github.com/pkg/errors" +) + +var ( + errInvalidArgCount = errors.New("invalid number of arguments") +) + +func main() { + gen := execgen{stdErr: os.Stderr} + if !gen.run(os.Args[1:]...) { + os.Exit(2) + } +} + +type execgen struct { + // useGoFmt runs the go fmt tool on code generated by execgen, if this setting + // is true. + useGoFmt bool + + // stdErr is the writer to which all standard error output will be redirected. + stdErr io.Writer + + // cmdLine stores the set of flags used to invoke the Execgen tool. + cmdLine *flag.FlagSet +} + +type generator func(io.Writer) error + +var generators = make(map[string]generator) + +func registerGenerator(g generator, filename string) { + if _, ok := generators[filename]; ok { + panic(fmt.Sprintf("%s generator already registered", filename)) + } + generators[filename] = g +} + +func (g *execgen) run(args ...string) bool { + // Parse command line. + g.cmdLine = flag.NewFlagSet("execgen", flag.ContinueOnError) + g.cmdLine.SetOutput(g.stdErr) + g.cmdLine.Usage = g.usage + g.cmdLine.BoolVar(&g.useGoFmt, "useGoFmt", true, "run go fmt on generated code") + err := g.cmdLine.Parse(args) + if err != nil { + return false + } + + // Get remaining args after any flags have been parsed. + args = g.cmdLine.Args() + if len(args) < 1 { + g.cmdLine.Usage() + g.reportError(errInvalidArgCount) + return false + } + + for _, out := range args { + _, file := filepath.Split(out) + gen := generators[file] + if gen == nil { + g.reportError(errors.Errorf("unrecognized filename: %s", file)) + return false + } + if err := g.generate(gen, out); err != nil { + g.reportError(err) + return false + } + } + + return true +} + +func (g *execgen) generate(genFunc generator, out string) error { + var buf bytes.Buffer + buf.WriteString("// Code generated by execgen; DO NOT EDIT.\n") + + err := genFunc(&buf) + if err != nil { + return err + } + + var b []byte + if g.useGoFmt { + + b, err = format.Source(buf.Bytes()) + if err != nil { + // Write out incorrect source for easier debugging. + b = buf.Bytes() + err = errors.Wrap(err, "Code formatting failed with Go parse error") + } + } else { + b = buf.Bytes() + } + + if err != nil { + // Ignore any write error if another error already occurred. + _ = g.writeOutputFile(b, out) + return err + } + return g.writeOutputFile(b, out) +} + +func (g *execgen) writeOutputFile(b []byte, out string) error { + file, err := os.Create(out) + if err != nil { + return err + } + defer file.Close() + + _, err = file.Write(b) + return err +} + +// usage is a replacement usage function for the flags package. +func (g *execgen) usage() { + fmt.Fprintf(g.stdErr, "Execgen is a tool for generating templated code related to ") + fmt.Fprintf(g.stdErr, "columnarized execution.\n\n") + + fmt.Fprintf(g.stdErr, "Usage:\n") + fmt.Fprintf(g.stdErr, "\texecgen [path]...\n\n") + + fmt.Fprintf(g.stdErr, "Supported filenames are:\n") + for filename := range generators { + fmt.Fprintf(g.stdErr, "\t%s\n", filename) + } + fmt.Fprintf(g.stdErr, "\n") + + fmt.Fprintf(g.stdErr, "Flags:\n") + g.cmdLine.PrintDefaults() + fmt.Fprintf(g.stdErr, "\n") +} + +func (g *execgen) reportError(err error) { + fmt.Fprintf(g.stdErr, "ERROR: %v\n", err) +} diff --git a/pkg/sql/exec/execgen/cmd/execgen/rowstovec_gen.go b/pkg/sql/exec/execgen/cmd/execgen/rowstovec_gen.go new file mode 100644 index 000000000000..59106d8cd2e4 --- /dev/null +++ b/pkg/sql/exec/execgen/cmd/execgen/rowstovec_gen.go @@ -0,0 +1,165 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package main + +import ( + "fmt" + "io" + "text/template" + + "github.com/cockroachdb/cockroach/pkg/sql/exec/types" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" +) + +const rowsToVecTemplate = ` +package exec + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/encoding" +) + +// EncDatumRowsToColVec converts one column from EncDatumRows to a column +// vector. columnIdx is the 0-based index of the column in the EncDatumRows. +func EncDatumRowsToColVec( + rows sqlbase.EncDatumRows, + vec ColVec, + columnIdx int, + columnType *sqlbase.ColumnType, + alloc *sqlbase.DatumAlloc, +) error { + nRows := uint16(len(rows)) + // TODO(solon): Make this chain of conditionals more efficient: either a + // switch statement or even better a lookup table on SemanticType. Also get + // rid of the somewhat dubious assumption that Width is unset (0) for column + // types where it does not apply. + {{range .}} + if columnType.SemanticType == sqlbase.{{.SemanticType}} && columnType.Width == {{.Width}} { + col := vec.{{.ExecType}}() + for i := uint16(0); i < nRows; i++ { + if rows[i][columnIdx].Datum == nil { + if err := rows[i][columnIdx].EnsureDecoded(columnType, alloc); err != nil { + return err + } + } + datum := rows[i][columnIdx].Datum + if datum == tree.DNull { + vec.SetNull(i) + } else { + {{if .HasSetMethod}} + col.Set(i, {{.DatumToPhysicalFn}}) + {{else}} + col[i] = {{.DatumToPhysicalFn}} + {{end}} + } + } + return nil + } + {{end}} + panic(fmt.Sprintf("Unsupported column type and width: %s, %d", columnType.SQLString(), columnType.Width)) +} +` + +// columnConversion defines a conversion from a sqlbase.ColumnType to an +// exec.ColVec. +type columnConversion struct { + // SemanticType is the semantic type of the ColumnType. + SemanticType string + // Width is the optional width of the ColumnType. + Width int32 + // ExecType is the exec.T to which we're converting. It should correspond to + // a method name on exec.ColVec. + ExecType string + // HasSetMethod is true if the ColVec is an interface with a Set method rather + // than just a slice. + HasSetMethod bool + // DatumToPhysicalFn is a stringified function for converting a datum to the + // physical type used in the column vector. + DatumToPhysicalFn string +} + +func genRowsToVec(wr io.Writer) error { + // Build the list of supported column conversions. + var columnConversions []columnConversion + for s, name := range sqlbase.ColumnType_SemanticType_name { + semanticType := sqlbase.ColumnType_SemanticType(s) + for _, width := range getWidths(semanticType) { + ct := sqlbase.ColumnType{SemanticType: semanticType, Width: width} + t := types.FromColumnType(ct) + if t == types.Unhandled { + continue + } + conversion := columnConversion{ + SemanticType: "ColumnType_" + name, + Width: width, + ExecType: t.String(), + // TODO(solon): Determine the following fields via reflection. + HasSetMethod: t == types.Bool || t == types.Bytes, + DatumToPhysicalFn: getDatumToPhysicalFn(ct), + } + columnConversions = append(columnConversions, conversion) + } + } + + tmpl, err := template.New("rowsToVec").Parse(rowsToVecTemplate) + if err != nil { + return err + } + return tmpl.Execute(wr, columnConversions) +} + +func init() { + registerGenerator(genRowsToVec, "rowstovec.og.go") +} + +// getWidths returns allowable ColumnType.Width values for the specified +// SemanticType. +func getWidths(semanticType sqlbase.ColumnType_SemanticType) []int32 { + if semanticType == sqlbase.ColumnType_INT { + return []int32{0, 8, 16, 32, 64} + } + return []int32{0} +} + +func getDatumToPhysicalFn(ct sqlbase.ColumnType) string { + switch ct.SemanticType { + case sqlbase.ColumnType_BOOL: + return "bool(*datum.(*tree.DBool))" + case sqlbase.ColumnType_BYTES: + return "encoding.UnsafeConvertStringToBytes(string(*datum.(*tree.DBytes)))" + case sqlbase.ColumnType_INT: + switch ct.Width { + case 8: + return "int8(*datum.(*tree.DInt))" + case 16: + return "int16(*datum.(*tree.DInt))" + case 32: + return "int32(*datum.(*tree.DInt))" + case 0, 64: + return "int64(*datum.(*tree.DInt))" + } + panic(fmt.Sprintf("unhandled INT width %d", ct.Width)) + case sqlbase.ColumnType_FLOAT: + return "float64(*datum.(*tree.DFloat))" + case sqlbase.ColumnType_OID: + return "int64(datum.(*tree.DOid).DInt)" + case sqlbase.ColumnType_STRING, sqlbase.ColumnType_NAME: + return "encoding.UnsafeConvertStringToBytes(string(*datum.(*tree.DString)))" + } + panic(fmt.Sprintf("unhandled ColumnType %s", ct.String())) +} diff --git a/pkg/sql/exec/rowstovec_test.go b/pkg/sql/exec/rowstovec_test.go new file mode 100644 index 000000000000..7fbe0a7e93a5 --- /dev/null +++ b/pkg/sql/exec/rowstovec_test.go @@ -0,0 +1,99 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package exec + +import ( + "reflect" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/exec/types" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" +) + +var alloc = sqlbase.DatumAlloc{} + +func TestEncDatumRowsToColVecBool(t *testing.T) { + // Test input: [[false, true], [true, false]] + rows := sqlbase.EncDatumRows{ + sqlbase.EncDatumRow{ + sqlbase.EncDatum{Datum: tree.DBoolFalse}, + sqlbase.EncDatum{Datum: tree.DBoolTrue}, + }, + sqlbase.EncDatumRow{ + sqlbase.EncDatum{Datum: tree.DBoolTrue}, + sqlbase.EncDatum{Datum: tree.DBoolFalse}, + }, + } + vec := newMemColumn(types.Bool, 2) + ct := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_BOOL} + + // Test converting column 0. + if err := EncDatumRowsToColVec(rows, vec, 0 /* columnIdx */, &ct, &alloc); err != nil { + t.Fatal(err) + } + expected := newMemColumn(types.Bool, 2) + expected.Bool().Set(0, false) + expected.Bool().Set(1, true) + if !reflect.DeepEqual(vec, expected) { + t.Errorf("expected vector %+v, got %+v", expected, vec) + } + + // Test converting column 1. + if err := EncDatumRowsToColVec(rows, vec, 1 /* columnIdx */, &ct, &alloc); err != nil { + t.Fatal(err) + } + expected.Bool().Set(0, true) + expected.Bool().Set(1, false) + if !reflect.DeepEqual(vec, expected) { + t.Errorf("expected vector %+v, got %+v", expected, vec) + } +} + +func TestEncDatumRowsToColVecInt16(t *testing.T) { + rows := sqlbase.EncDatumRows{ + sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDInt(17)}}, + sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDInt(42)}}, + } + vec := newMemColumn(types.Int16, 2) + ct := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_INT, Width: 16} + if err := EncDatumRowsToColVec(rows, vec, 0 /* columnIdx */, &ct, &alloc); err != nil { + t.Fatal(err) + } + expected := newMemColumn(types.Int16, 2) + expected.Int16()[0] = 17 + expected.Int16()[1] = 42 + if !reflect.DeepEqual(vec, expected) { + t.Errorf("expected vector %+v, got %+v", expected, vec) + } +} + +func TestEncDatumRowsToColVecString(t *testing.T) { + rows := sqlbase.EncDatumRows{ + sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDString("foo")}}, + sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDString("bar")}}, + } + vec := newMemColumn(types.Bytes, 2) + ct := sqlbase.ColumnType{SemanticType: sqlbase.ColumnType_STRING} + if err := EncDatumRowsToColVec(rows, vec, 0 /* columnIdx */, &ct, &alloc); err != nil { + t.Fatal(err) + } + expected := newMemColumn(types.Bytes, 2) + expected.Bytes().Set(0, []byte("foo")) + expected.Bytes().Set(1, []byte("bar")) + if !reflect.DeepEqual(vec, expected) { + t.Errorf("expected vector %+v, got %+v", expected, vec) + } +} diff --git a/pkg/sql/exec/types/t_string.go b/pkg/sql/exec/types/t_string.go new file mode 100644 index 000000000000..8039d8666fac --- /dev/null +++ b/pkg/sql/exec/types/t_string.go @@ -0,0 +1,16 @@ +// Code generated by "stringer -type=T"; DO NOT EDIT. + +package types + +import "strconv" + +const _T_name = "BoolBytesInt8Int16Int32Int64Float32Float64Unhandled" + +var _T_index = [...]uint8{0, 4, 9, 13, 18, 23, 28, 35, 42, 51} + +func (i T) String() string { + if i < 0 || i >= T(len(_T_index)-1) { + return "T(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _T_name[_T_index[i]:_T_index[i+1]] +} diff --git a/pkg/sql/exec/types/types.go b/pkg/sql/exec/types/types.go index 296252224d6e..e2ac4a4c611e 100644 --- a/pkg/sql/exec/types/types.go +++ b/pkg/sql/exec/types/types.go @@ -24,6 +24,7 @@ import ( // column type. type T int +//go:generate stringer -type=T const ( // Bool is a column of type bool Bool T = iota