Skip to content

Commit

Permalink
Merge #56660
Browse files Browse the repository at this point in the history
56660: sql: use catalog.TableColSet instead of map[descpb.ColumnID]struct{} r=mgartner a=mgartner

#### sql: move TableColSet from schemaexpr package to catalog

The `TableColSet` type describes a set of columns of a table. This
commit moves the type to the `catalog` package because it its usage is
not specific to the functionality in the `schemaexpr` package.

Release note: None

#### sql: use catalog.TableColSet instead of map[descpb.ColumnID]struct{}

This commit converts all instances where a set of column IDs was
implemented with a `map[descpb.ColumnID]struct{}` with the more
efficient `catalog.TableColSet`, a wrapper around `util.FastIntSet`.

Release note: None


Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
craig[bot] and mgartner committed Nov 18, 2020
2 parents 6175638 + c3dd9ab commit 10909f2
Show file tree
Hide file tree
Showing 23 changed files with 91 additions and 99 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_avro.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,7 @@ func (a *avroConsumer) FillDatums(
// Set any nil datums to DNull (in case native
// record didn't have the value set at all)
for i := range conv.Datums {
if _, isTargetCol := conv.IsTargetCol[i]; isTargetCol && conv.Datums[i] == nil {
if conv.TargetColOrds.Contains(i) && conv.Datums[i] == nil {
if a.strict {
return fmt.Errorf("field %s was not set in the avro import", conv.VisibleCols[i].Name)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_csv.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ func (c *csvRowConsumer) FillDatums(
for i, field := range record {
// Skip over record entries corresponding to columns not in the target
// columns specified by the user.
if _, ok := conv.IsTargetCol[i]; !ok {
if !conv.TargetColOrds.Contains(i) {
continue
}

Expand Down
13 changes: 7 additions & 6 deletions pkg/ccl/importccl/read_import_pgdump.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -672,21 +673,21 @@ func (m *pgDumpReader) readFile(
var targetColMapIdx []int
if len(i.Columns) != 0 {
targetColMapIdx = make([]int, len(i.Columns))
conv.IsTargetCol = make(map[int]struct{}, len(i.Columns))
conv.TargetColOrds = util.FastIntSet{}
for j := range i.Columns {
colName := string(i.Columns[j])
idx, ok := m.colMap[conv][colName]
if !ok {
return errors.Newf("targeted column %q not found", colName)
}
conv.IsTargetCol[idx] = struct{}{}
conv.TargetColOrds.Add(idx)
targetColMapIdx[j] = idx
}
// For any missing columns, fill those to NULL.
// These will get filled in with the correct default / computed expression
// provided conv.IsTargetCol is not set for the given column index.
for idx := range conv.VisibleCols {
if _, ok := conv.IsTargetCol[idx]; !ok {
if !conv.TargetColOrds.Contains(idx) {
conv.Datums[idx] = tree.DNull
}
}
Expand Down Expand Up @@ -735,14 +736,14 @@ func (m *pgDumpReader) readFile(
var targetColMapIdx []int
if conv != nil {
targetColMapIdx = make([]int, len(i.Columns))
conv.IsTargetCol = make(map[int]struct{}, len(i.Columns))
conv.TargetColOrds = util.FastIntSet{}
for j := range i.Columns {
colName := string(i.Columns[j])
idx, ok := m.colMap[conv][colName]
if !ok {
return errors.Newf("targeted column %q not found", colName)
}
conv.IsTargetCol[idx] = struct{}{}
conv.TargetColOrds.Add(idx)
targetColMapIdx[j] = idx
}
}
Expand All @@ -768,7 +769,7 @@ func (m *pgDumpReader) readFile(
}
switch row := row.(type) {
case copyData:
if expected, got := len(conv.IsTargetCol), len(row); expected != got {
if expected, got := conv.TargetColOrds.Len(), len(row); expected != got {
return makeRowErr("", count, pgcode.Syntax,
"expected %d values, got %d", expected, got)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -465,7 +465,7 @@ func (ib *IndexBackfiller) InitForDistributedUse(

evalCtx := flowCtx.NewEvalCtx()
var predicates map[descpb.IndexID]tree.TypedExpr
var predicateRefColIDs schemaexpr.TableColSet
var predicateRefColIDs catalog.TableColSet

// Install type metadata in the target descriptors, as well as resolve any
// user defined types in partial index predicate expressions.
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
"descriptor.go",
"errors.go",
"table_col_map.go",
"table_col_set.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog",
visibility = ["//visibility:public"],
Expand All @@ -34,6 +35,7 @@ go_test(
srcs = [
"dep_test.go",
"descriptor_test.go",
"table_col_set_test.go",
],
embed = [":catalog"],
deps = [
Expand All @@ -42,6 +44,7 @@ go_test(
"//pkg/sql/catalog/schemadesc",
"//pkg/sql/catalog/tabledesc",
"//pkg/testutils/buildutil",
"//pkg/util",
"//pkg/util/leaktest",
"//vendor/github.com/cockroachdb/redact",
"//vendor/github.com/stretchr/testify/require",
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/catalog/colinfo/column_resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,8 @@ func ProcessTargetColumns(
return nil, nil
}

var colIDSet catalog.TableColSet
cols := make([]descpb.ColumnDescriptor, len(nameList))
colIDSet := make(map[descpb.ColumnID]struct{}, len(nameList))
for i, colName := range nameList {
var col *descpb.ColumnDescriptor
var err error
Expand All @@ -56,11 +56,11 @@ func ProcessTargetColumns(
return nil, err
}

if _, ok := colIDSet[col.ID]; ok {
if colIDSet.Contains(col.ID) {
return nil, pgerror.Newf(pgcode.Syntax,
"multiple assignments to the same column %q", &nameList[i])
}
colIDSet[col.ID] = struct{}{}
colIDSet.Add(col.ID)
cols[i] = *col
}

Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/catalog/schemaexpr/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ go_library(
"expr_filter.go",
"partial_index.go",
"select_name_resolution.go",
"table_col_set.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr",
visibility = ["//visibility:public"],
Expand All @@ -28,7 +27,6 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/types",
"//pkg/util",
"//vendor/github.com/cockroachdb/errors",
],
)
Expand All @@ -40,7 +38,6 @@ go_test(
"column_test.go",
"expr_test.go",
"partial_index_test.go",
"table_col_set_test.go",
"testutils_test.go",
],
embed = [":schemaexpr"],
Expand All @@ -53,6 +50,5 @@ go_test(
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util",
],
)
4 changes: 2 additions & 2 deletions pkg/sql/catalog/schemaexpr/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -234,8 +234,8 @@ func (d *dummyColumn) ResolvedType() *types.T {
// descriptor, replaceColumnVars errs with pgcode.UndefinedColumn.
func replaceColumnVars(
desc catalog.TableDescriptor, rootExpr tree.Expr,
) (tree.Expr, TableColSet, error) {
var colIDs TableColSet
) (tree.Expr, catalog.TableColSet, error) {
var colIDs catalog.TableColSet

newExpr, err := tree.SimpleVisit(rootExpr, func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) {
vBase, ok := expr.(tree.VarName)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/schemaexpr/computed_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ func (v *ComputedColumnValidator) Validate(d *tree.ColumnTableDef) error {
)
}

var depColIDs TableColSet
var depColIDs catalog.TableColSet
// First, check that no column in the expression is a computed column.
err := iterColDescriptors(v.desc, d.Computed.Expr, func(c *descpb.ColumnDescriptor) error {
if c.IsComputed() {
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/catalog/schemaexpr/default_exprs.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,9 +90,9 @@ func ProcessColumnSet(
tableDesc catalog.TableDescriptor,
inSet func(*descpb.ColumnDescriptor) bool,
) []descpb.ColumnDescriptor {
colIDSet := make(map[descpb.ColumnID]struct{}, len(cols))
var colIDSet catalog.TableColSet
for i := range cols {
colIDSet[cols[i].ID] = struct{}{}
colIDSet.Add(cols[i].ID)
}

// Add all public or columns in DELETE_AND_WRITE_ONLY state
Expand All @@ -101,8 +101,8 @@ func ProcessColumnSet(
for i := range writable {
col := &writable[i]
if inSet(col) {
if _, ok := colIDSet[col.ID]; !ok {
colIDSet[col.ID] = struct{}{}
if !colIDSet.Contains(col.ID) {
colIDSet.Add(col.ID)
cols = append(cols, *col)
}
}
Expand Down
10 changes: 6 additions & 4 deletions pkg/sql/catalog/schemaexpr/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,8 @@ func DequalifyAndValidateExpr(
semaCtx *tree.SemaContext,
maxVolatility tree.Volatility,
tn *tree.TableName,
) (string, TableColSet, error) {
var colIDs TableColSet
) (string, catalog.TableColSet, error) {
var colIDs catalog.TableColSet
sourceInfo := colinfo.NewSourceInfoForSingleTable(
*tn, colinfo.ResultColumnsFromColDescs(
desc.GetID(),
Expand Down Expand Up @@ -80,8 +80,10 @@ func DequalifyAndValidateExpr(
}

// ExtractColumnIDs returns the set of column IDs within the given expression.
func ExtractColumnIDs(desc catalog.TableDescriptor, rootExpr tree.Expr) (TableColSet, error) {
var colIDs TableColSet
func ExtractColumnIDs(
desc catalog.TableDescriptor, rootExpr tree.Expr,
) (catalog.TableColSet, error) {
var colIDs catalog.TableColSet

_, err := tree.SimpleVisit(rootExpr, func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) {
vBase, ok := expr.(tree.VarName)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/schemaexpr/partial_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func MakePartialIndexExprs(
tableDesc catalog.TableDescriptor,
evalCtx *tree.EvalContext,
semaCtx *tree.SemaContext,
) (_ map[descpb.IndexID]tree.TypedExpr, refColIDs TableColSet, _ error) {
) (_ map[descpb.IndexID]tree.TypedExpr, refColIDs catalog.TableColSet, _ error) {
// If none of the indexes are partial indexes, return early.
partialIndexCount := 0
for i := range indexes {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package schemaexpr
package catalog

import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -41,14 +41,18 @@ func (s TableColSet) Empty() bool { return s.set.Empty() }
// Len returns the number of the columns in the set.
func (s TableColSet) Len() int { return s.set.Len() }

// Next returns the first value in the set which is >= startVal. If there is no
// value, the second return value is false.
func (s TableColSet) Next(startVal descpb.ColumnID) (descpb.ColumnID, bool) {
c, ok := s.set.Next(int(startVal))
return descpb.ColumnID(c), ok
}

// ForEach calls a function for each column in the set (in increasing order).
func (s TableColSet) ForEach(f func(col descpb.ColumnID)) {
s.set.ForEach(func(i int) { f(descpb.ColumnID(i)) })
}

// UnionWith adds all the columns from rhs to this set.
func (s *TableColSet) UnionWith(rhs TableColSet) { s.set.UnionWith(rhs.set) }

// Ordered returns a slice with all the descpb.ColumnIDs in the set, in
// increasing order.
func (s TableColSet) Ordered() []descpb.ColumnID {
Expand All @@ -62,6 +66,9 @@ func (s TableColSet) Ordered() []descpb.ColumnID {
return result
}

// UnionWith adds all the columns from rhs to this set.
func (s *TableColSet) UnionWith(rhs TableColSet) { s.set.UnionWith(rhs.set) }

// String returns a list representation of elements. Sequential runs of positive
// numbers are shown as ranges. For example, for the set {1, 2, 3 5, 6, 10},
// the output is "(1-3,5,6,10)".
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package schemaexpr
package catalog

import (
"testing"
Expand Down
Loading

0 comments on commit 10909f2

Please sign in to comment.