diff --git a/Gopkg.lock b/Gopkg.lock index 71a0d8270d05..af17ce779764 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -1339,14 +1339,14 @@ [[projects]] branch = "master" - digest = "1:b66436e3c460ee4a9fab4f7dc473c36c51d3263e04838756378e72cc90058b6c" + digest = "1:6830d356b8696fd4e7f09b57245dff696d19de8449ba589866e3783e80347e3e" name = "go.etcd.io/etcd" packages = [ "raft", "raft/raftpb", ] pruneopts = "UT" - revision = "dac8c6fcc05ba42a8032d5b720f6c1704965c269" + revision = "b42b39446bc1b563ba58aceda53b6ecad87d73f9" [[projects]] digest = "1:f163a34487229f36dfdb298191d8e17c0e3e6a899aa2cddb020f2ac61ca364ab" diff --git a/Makefile b/Makefile index 2a85265814d5..2bdc243f7511 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 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 4936694555f2..9db78f852a9a 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1624,3 +1624,61 @@ query OT SELECT typ.oid, typ.typname FROM pg_attribute att JOIN pg_type typ ON atttypid=typ.oid WHERE attrelid='coltab'::regclass AND attname='a' ---- 25 text + +subtest 31545 + +# Test an index of 2 referencing an index of 2. +statement ok +CREATE TABLE a ( + id_a_1 INT UNIQUE, + id_a_2 INT, + PRIMARY KEY (id_a_1, id_a_2) +) + +statement ok +CREATE TABLE b ( + id_b_1 INT, + id_b_2 INT, + PRIMARY KEY (id_b_1, id_b_2), + CONSTRAINT my_fkey FOREIGN KEY (id_b_1, id_b_2) REFERENCES a (id_a_1, id_a_2) +) + +query TT colnames +SELECT conkey, confkey FROM pg_catalog.pg_constraint WHERE conname = 'my_fkey' +---- +conkey confkey +{1,2} {1,2} + +# Test an index of 3 referencing an index of 2. +statement ok +DROP TABLE b; +CREATE TABLE b ( + id_b_1 INT, + id_b_2 INT, + id_b_3 INT, + PRIMARY KEY (id_b_1, id_b_2, id_b_3), + CONSTRAINT my_fkey FOREIGN KEY (id_b_1, id_b_2) REFERENCES a (id_a_1, id_a_2) +) + +query TT colnames +SELECT conkey, confkey FROM pg_catalog.pg_constraint WHERE conname = 'my_fkey' +---- +conkey confkey +{1,2} {1,2} + +# Test an index of 3 referencing an index of 1. +statement ok +DROP TABLE b; +CREATE TABLE b ( + id_b_1 INT, + id_b_2 INT, + id_b_3 INT, + PRIMARY KEY (id_b_1, id_b_2, id_b_3), + CONSTRAINT my_fkey FOREIGN KEY (id_b_1) REFERENCES a (id_a_1) +) + +query TT colnames +SELECT conkey, confkey FROM pg_catalog.pg_constraint WHERE conname = 'my_fkey' +---- +conkey confkey +{1} {1} diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index b87027ae93b3..ecb8e342b30a 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -680,7 +680,21 @@ CREATE TABLE pg_catalog.pg_constraint ( confupdtype = fkActionNone confdeltype = fkActionNone confmatchtype = fkMatchTypeSimple - if conkey, err = colIDArrayToDatum(con.Index.ColumnIDs); err != nil { + columnIDs := con.Index.ColumnIDs + if int(con.FK.SharedPrefixLen) > len(columnIDs) { + return errors.Errorf( + "For foreign key %q's shared prefix len (%d) is greater than the number of columns "+ + "in the index (%d). This might be an indication of inconsistency.", + con.FK.Name, + con.FK.SharedPrefixLen, + int32(len(columnIDs)), + ) + } + sharedPrefixLen := len(columnIDs) + if int(con.FK.SharedPrefixLen) > 0 { + sharedPrefixLen = int(con.FK.SharedPrefixLen) + } + if conkey, err = colIDArrayToDatum(columnIDs[:sharedPrefixLen]); err != nil { return err } if confkey, err = colIDArrayToDatum(con.ReferencedIndex.ColumnIDs); err != nil { diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 0222d118a75a..c7ed7371b478 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -655,6 +655,8 @@ type InterleaveDescriptor_Ancestor struct { // grandparent. Thus, the sum of SharedPrefixLens in the components of an // InterleaveDescriptor is never more than the number of fields in the index // being interleaved. + // In cockroach 1.0, this value did not exist and thus a check for > 0 + // must be performed prior to its use. SharedPrefixLen uint32 `protobuf:"varint,3,opt,name=shared_prefix_len,json=sharedPrefixLen" json:"shared_prefix_len"` } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index cc285625362e..bc3a90322d15 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -285,6 +285,8 @@ message InterleaveDescriptor { // grandparent. Thus, the sum of SharedPrefixLens in the components of an // InterleaveDescriptor is never more than the number of fields in the index // being interleaved. + // In cockroach 1.0, this value did not exist and thus a check for > 0 + // must be performed prior to its use. optional uint32 shared_prefix_len = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "SharedPrefixLen"]; } diff --git a/vendor b/vendor index e82d6138ecdb..50f622b97b35 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit e82d6138ecdb655bf5eb2f353346615489dc7a9a +Subproject commit 50f622b97b353426537455f5be003d15234d23f0