From c106c32b8106e2be42193ac6e329f5bc9979e805 Mon Sep 17 00:00:00 2001 From: Solon Gordon Date: Thu, 11 Oct 2018 17:13:11 -0400 Subject: [PATCH 1/3] 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 From b6dae2fee3b69abcaaa82a2f29cc6c4afcb938cf Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Mon, 22 Oct 2018 21:39:24 +0200 Subject: [PATCH 2/3] storage: pick up fix for Raft uncommitted entry size tracking The tracking of the uncommitted portion of the log had a bug where it wasn't releasing everything as it should've. As a result, over time, all proposals would be dropped. We're hitting this way earlier in our import tests, which propose large proposals. As an intentional implementation detail, a proposal that itself exceeds the max uncommitted log size is allowed only if the uncommitted log is empty. Due to the leak, we weren't ever hitting this case and so AddSSTable commands were often dropped indefinitely. Fixes #31184. Fixes #28693. Fixes #31642. Optimistically: Fixes #31675. Fixes #31654. Fixes #31446. Release note: None --- Gopkg.lock | 4 ++-- vendor | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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/vendor b/vendor index e82d6138ecdb..50f622b97b35 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit e82d6138ecdb655bf5eb2f353346615489dc7a9a +Subproject commit 50f622b97b353426537455f5be003d15234d23f0 From 0fd79ec354d774d8a5d62354b228263a392ea287 Mon Sep 17 00:00:00 2001 From: Bram Gruneir Date: Thu, 18 Oct 2018 16:57:35 -0400 Subject: [PATCH 3/3] sql: fix pg_catalog.pg_constraint's confkey column Prior to this patch, all columns in the index were included instead of only the ones being used in the foreign key reference. Fixes #31545. Release note (bug fix): Fix pg_catalog.pg_constraint's confkey column from including columns that were not involved in the foreign key reference. --- .../logictest/testdata/logic_test/pg_catalog | 58 +++++++++++++++++++ pkg/sql/pg_catalog.go | 16 ++++- pkg/sql/sqlbase/structured.pb.go | 2 + pkg/sql/sqlbase/structured.proto | 2 + 4 files changed, 77 insertions(+), 1 deletion(-) 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"]; }