Skip to content

Commit

Permalink
Merge #39569
Browse files Browse the repository at this point in the history
39569: col: move types,coldata,colserde under new top-level col package r=yuzefovich a=danhhz

pkg/sql/exec/types -> pkg/col/coltypes
pkg/sql/exec/coldata -> pkg/col/coldata
pkg/sql/exec/colserde -> pkg/col/colserde
pkg/sql/exec/types/conv -> pkg/sql/exec/typeconv

For three reasons:
1) Make it obvious which of our packages are an independent columnar
   data library and which are tied to our sql execution (colrpc).
2) In the process, rename types to coltypes to clear up importing
   confusion.
3) Allow workload to blanket ban deps on anything in pkg/sql.

Release note: None

Co-authored-by: Daniel Harrison <[email protected]>
  • Loading branch information
craig[bot] and danhhz committed Aug 14, 2019
2 parents bad5e2e + 85caac9 commit 3110746
Show file tree
Hide file tree
Showing 158 changed files with 1,467 additions and 1,437 deletions.
4 changes: 2 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -782,9 +782,9 @@ PROTOBUF_TARGETS := bin/.go_protobuf_sources bin/.gw_protobuf_sources bin/.cpp_p
DOCGEN_TARGETS := bin/.docgen_bnfs bin/.docgen_functions

EXECGEN_TARGETS = \
pkg/col/coldata/vec.eg.go \
pkg/sql/exec/any_not_null_agg.eg.go \
pkg/sql/exec/avg_agg.eg.go \
pkg/sql/exec/coldata/vec.eg.go \
pkg/sql/exec/const.eg.go \
pkg/sql/exec/distinct.eg.go \
pkg/sql/exec/hashjoiner.eg.go \
Expand Down Expand Up @@ -1454,9 +1454,9 @@ settings-doc-gen := $(if $(filter buildshort,$(MAKECMDGOALS)),$(COCKROACHSHORT),
$(SETTINGS_DOC_PAGE): $(settings-doc-gen)
@$(settings-doc-gen) gen settings-list --format=html > $@

pkg/col/coldata/vec.eg.go: pkg/col/coldata/vec_tmpl.go
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/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
pkg/sql/exec/hashjoiner.eg.go: pkg/sql/exec/hashjoiner_tmpl.go
Expand Down
32 changes: 14 additions & 18 deletions pkg/ccl/importccl/read_import_workload.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,15 @@ import (

"github.com/cockroachdb/apd"
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/exec/coldata"
"github.com/cockroachdb/cockroach/pkg/sql/exec/types"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
sqltypes "github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/bufalloc"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -58,42 +58,38 @@ func (w *workloadReader) inputFinished(ctx context.Context) {
// makeDatumFromColOffset tries to fast-path a few workload-generated types into
// directly datums, to dodge making a string and then the parsing it.
func makeDatumFromColOffset(
alloc *sqlbase.DatumAlloc,
hint *sqltypes.T,
evalCtx *tree.EvalContext,
col coldata.Vec,
rowIdx int,
alloc *sqlbase.DatumAlloc, hint *types.T, evalCtx *tree.EvalContext, col coldata.Vec, rowIdx int,
) (tree.Datum, error) {
if col.Nulls().NullAt64(uint64(rowIdx)) {
return tree.DNull, nil
}
switch col.Type() {
case types.Bool:
case coltypes.Bool:
return tree.MakeDBool(tree.DBool(col.Bool()[rowIdx])), nil
case types.Int64:
case coltypes.Int64:
switch hint.Family() {
case sqltypes.IntFamily:
case types.IntFamily:
return alloc.NewDInt(tree.DInt(col.Int64()[rowIdx])), nil
case sqltypes.DecimalFamily:
case types.DecimalFamily:
d := *apd.New(col.Int64()[rowIdx], 0)
return alloc.NewDDecimal(tree.DDecimal{Decimal: d}), nil
}
case types.Float64:
case coltypes.Float64:
switch hint.Family() {
case sqltypes.FloatFamily:
case types.FloatFamily:
return alloc.NewDFloat(tree.DFloat(col.Float64()[rowIdx])), nil
case sqltypes.DecimalFamily:
case types.DecimalFamily:
var d apd.Decimal
if _, err := d.SetFloat64(col.Float64()[rowIdx]); err != nil {
return nil, err
}
return alloc.NewDDecimal(tree.DDecimal{Decimal: d}), nil
}
case types.Bytes:
case coltypes.Bytes:
switch hint.Family() {
case sqltypes.BytesFamily:
case types.BytesFamily:
return alloc.NewDBytes(tree.DBytes(col.Bytes().Get(rowIdx))), nil
case sqltypes.StringFamily:
case types.StringFamily:
data := col.Bytes().Get(rowIdx)
str := *(*string)(unsafe.Pointer(&data))
return alloc.NewDString(tree.DString(str)), nil
Expand Down
12 changes: 6 additions & 6 deletions pkg/ccl/workloadccl/allccl/all_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
_ "github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/ccl/workloadccl"
"github.com/cockroachdb/cockroach/pkg/sql/exec/coldata"
"github.com/cockroachdb/cockroach/pkg/sql/exec/types"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util"
Expand Down Expand Up @@ -192,7 +192,7 @@ func hashTableInitialData(
data.FillBatch(batchIdx, b, a)
for _, col := range b.ColVecs() {
switch col.Type() {
case types.Bool:
case coltypes.Bool:
for _, x := range col.Bool()[:b.Length()] {
if x {
scratch[0] = 1
Expand All @@ -201,18 +201,18 @@ func hashTableInitialData(
}
_, _ = h.Write(scratch[:1])
}
case types.Int64:
case coltypes.Int64:
for _, x := range col.Int64()[:b.Length()] {
binary.LittleEndian.PutUint64(scratch[:8], uint64(x))
_, _ = h.Write(scratch[:8])
}
case types.Float64:
case coltypes.Float64:
for _, x := range col.Float64()[:b.Length()] {
bits := math.Float64bits(x)
binary.LittleEndian.PutUint64(scratch[:8], bits)
_, _ = h.Write(scratch[:8])
}
case types.Bytes:
case coltypes.Bytes:
colBytes := col.Bytes()
for i := 0; i < int(b.Length()); i++ {
_, _ = h.Write(colBytes.Get(i))
Expand Down
1 change: 1 addition & 0 deletions pkg/col/coldata/.gitignore
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
vec.eg.go
16 changes: 8 additions & 8 deletions pkg/sql/exec/coldata/batch.go → pkg/col/coldata/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ import (
"fmt"
"math"

"github.com/cockroachdb/cockroach/pkg/sql/exec/types"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
)

// Batch is the type that columnar operators receive and produce. It
Expand All @@ -39,12 +39,12 @@ type Batch interface {
// SetSelection sets whether this batch is using its selection vector or not.
SetSelection(bool)
// AppendCol appends a Vec with the specified type to this batch.
AppendCol(types.T)
AppendCol(coltypes.T)
// Reset modifies the caller in-place to have the given length and columns
// with the given types. If it's possible, Reset will reuse the existing
// with the given coltypes. If it's possible, Reset will reuse the existing
// columns and allocations, invalidating existing references to the Batch or
// its Vecs. However, Reset does _not_ zero out the column data.
Reset(types []types.T, length int)
Reset(types []coltypes.T, length int)
}

var _ Batch = &MemBatch{}
Expand All @@ -55,13 +55,13 @@ const BatchSize = 1024

// NewMemBatch allocates a new in-memory Batch.
// TODO(jordan): pool these allocations.
func NewMemBatch(types []types.T) Batch {
func NewMemBatch(types []coltypes.T) Batch {
return NewMemBatchWithSize(types, BatchSize)
}

// NewMemBatchWithSize allocates a new in-memory Batch with the given column
// size. Use for operators that have a precisely-sized output batch.
func NewMemBatchWithSize(types []types.T, size int) Batch {
func NewMemBatchWithSize(types []coltypes.T, size int) Batch {
if max := math.MaxUint16; size > max {
panic(fmt.Sprintf(`batches cannot have length larger than %d; requested %d`, max, size))
}
Expand Down Expand Up @@ -127,12 +127,12 @@ func (m *MemBatch) SetLength(n uint16) {
}

// AppendCol implements the Batch interface.
func (m *MemBatch) AppendCol(t types.T) {
func (m *MemBatch) AppendCol(t coltypes.T) {
m.b = append(m.b, NewMemColumn(t, BatchSize))
}

// Reset implements the Batch interface.
func (m *MemBatch) Reset(types []types.T, length int) {
func (m *MemBatch) Reset(types []coltypes.T, length int) {
// The columns are always sized the same as the selection vector, so use it as
// a shortcut for the capacity (like a go slice, the batch's `Length` could be
// shorter than the capacity). We could be more defensive and type switch
Expand Down
File renamed without changes.
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ func TestNoLinkForbidden(t *testing.T) {
defer leaktest.AfterTest(t)()

buildutil.VerifyNoImports(t,
"github.com/cockroachdb/cockroach/pkg/sql/exec/coldata", true,
"github.com/cockroachdb/cockroach/pkg/col/coldata", true,
[]string{
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase",
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree",
Expand Down
File renamed without changes.
File renamed without changes.
File renamed without changes.
36 changes: 18 additions & 18 deletions pkg/sql/exec/coldata/vec.go → pkg/col/coldata/vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ import (
"fmt"

"github.com/cockroachdb/apd"
"github.com/cockroachdb/cockroach/pkg/sql/exec/types"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
)

// column is an interface that represents a raw array of a Go native type.
Expand All @@ -23,7 +23,7 @@ type column interface{}
// AppendArgs represents the arguments passed in to Vec.Append.
type AppendArgs struct {
// ColType is the type of both the destination and source slices.
ColType types.T
ColType coltypes.T
// Src is the data being appended.
Src Vec
// Sel is an optional slice specifying indices to append to the destination
Expand All @@ -43,7 +43,7 @@ type AppendArgs struct {
// CopyArgs represents the arguments passed in to Vec.Copy.
type CopyArgs struct {
// ColType is the type of both the destination and source slices.
ColType types.T
ColType coltypes.T
// Src is the data being copied.
Src Vec
// Sel is an optional slice specifying indices to copy to the destination
Expand Down Expand Up @@ -76,7 +76,7 @@ type CopyArgs struct {
// Go native types.
type Vec interface {
// Type returns the type of data stored in this Vec.
Type() types.T
Type() coltypes.T

// TODO(jordan): is a bitmap or slice of bools better?
// Bool returns a bool list.
Expand Down Expand Up @@ -127,11 +127,11 @@ type Vec interface {

// Slice returns a new Vec representing a slice of the current Vec from
// [start, end).
Slice(colType types.T, start uint64, end uint64) Vec
Slice(colType coltypes.T, start uint64, end uint64) Vec

// PrettyValueAt returns a "pretty"value for the idx'th value in this Vec.
// It uses the reflect package and is not suitable for calling in hot paths.
PrettyValueAt(idx uint16, colType types.T) string
PrettyValueAt(idx uint16, colType coltypes.T) string

// MaybeHasNulls returns true if the column possibly has any null values, and
// returns false if the column definitely has no null values.
Expand All @@ -149,40 +149,40 @@ var _ Vec = &memColumn{}
// memColumn is a simple pass-through implementation of Vec that just casts
// a generic interface{} to the proper type when requested.
type memColumn struct {
t types.T
t coltypes.T
col column
nulls Nulls
}

// NewMemColumn returns a new memColumn, initialized with a length.
func NewMemColumn(t types.T, n int) Vec {
func NewMemColumn(t coltypes.T, n int) Vec {
nulls := NewNulls(n)

switch t {
case types.Bool:
case coltypes.Bool:
return &memColumn{t: t, col: make([]bool, n), nulls: nulls}
case types.Bytes:
case coltypes.Bytes:
return &memColumn{t: t, col: NewBytes(n), nulls: nulls}
case types.Int8:
case coltypes.Int8:
return &memColumn{t: t, col: make([]int8, n), nulls: nulls}
case types.Int16:
case coltypes.Int16:
return &memColumn{t: t, col: make([]int16, n), nulls: nulls}
case types.Int32:
case coltypes.Int32:
return &memColumn{t: t, col: make([]int32, n), nulls: nulls}
case types.Int64:
case coltypes.Int64:
return &memColumn{t: t, col: make([]int64, n), nulls: nulls}
case types.Float32:
case coltypes.Float32:
return &memColumn{t: t, col: make([]float32, n), nulls: nulls}
case types.Float64:
case coltypes.Float64:
return &memColumn{t: t, col: make([]float64, n), nulls: nulls}
case types.Decimal:
case coltypes.Decimal:
return &memColumn{t: t, col: make([]apd.Decimal, n), nulls: nulls}
default:
panic(fmt.Sprintf("unhandled type %s", t))
}
}

func (m *memColumn) Type() types.T {
func (m *memColumn) Type() coltypes.T {
return m.t
}

Expand Down
18 changes: 9 additions & 9 deletions pkg/sql/exec/coldata/vec_test.go → pkg/col/coldata/vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ package coldata
import (
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/exec/types"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/require"
Expand All @@ -24,7 +24,7 @@ func TestMemColumnSlice(t *testing.T) {

rng, _ := randutil.NewPseudoRand()

c := NewMemColumn(types.Int64, BatchSize)
c := NewMemColumn(coltypes.Int64, BatchSize)

ints := c.Int64()
for i := uint16(0); i < BatchSize; i++ {
Expand All @@ -42,7 +42,7 @@ func TestMemColumnSlice(t *testing.T) {
endSlice = uint16(1 + rng.Intn(BatchSize))
}

slice := c.Slice(types.Int64, uint64(startSlice), uint64(endSlice))
slice := c.Slice(coltypes.Int64, uint64(startSlice), uint64(endSlice))
sliceInts := slice.Int64()
// Verify that every other value is null.
for i, j := startSlice, uint16(0); i < endSlice; i, j = i+1, j+1 {
Expand Down Expand Up @@ -112,7 +112,7 @@ func TestNullRanges(t *testing.T) {
},
}

c := NewMemColumn(types.Int64, BatchSize)
c := NewMemColumn(coltypes.Int64, BatchSize)
for _, tc := range tcs {
c.Nulls().UnsetNulls()
c.Nulls().SetNullRange(tc.start, tc.end)
Expand All @@ -133,7 +133,7 @@ func TestNullRanges(t *testing.T) {

func TestAppend(t *testing.T) {
// TODO(asubiotto): Test nulls.
const typ = types.Int64
const typ = coltypes.Int64

src := NewMemColumn(typ, BatchSize)
sel := make([]uint16, len(src.Int64()))
Expand Down Expand Up @@ -217,7 +217,7 @@ func TestAppend(t *testing.T) {

func TestCopy(t *testing.T) {
// TODO(asubiotto): Test nulls.
const typ = types.Int64
const typ = coltypes.Int64

src := NewMemColumn(typ, BatchSize)
srcInts := src.Int64()
Expand Down Expand Up @@ -300,7 +300,7 @@ func TestCopy(t *testing.T) {
}

func TestCopyNulls(t *testing.T) {
const typ = types.Int64
const typ = coltypes.Int64

// Set up the destination vector.
dst := NewMemColumn(typ, BatchSize)
Expand Down Expand Up @@ -354,7 +354,7 @@ func TestCopyNulls(t *testing.T) {
}

func BenchmarkAppend(b *testing.B) {
const typ = types.Int64
const typ = coltypes.Int64

src := NewMemColumn(typ, BatchSize)
sel := make([]uint16, len(src.Int64()))
Expand Down Expand Up @@ -392,7 +392,7 @@ func BenchmarkAppend(b *testing.B) {
}

func BenchmarkCopy(b *testing.B) {
const typ = types.Int64
const typ = coltypes.Int64

src := NewMemColumn(typ, BatchSize)
sel := make([]uint16, len(src.Int64()))
Expand Down
Loading

0 comments on commit 3110746

Please sign in to comment.