Commit 84c0a3e9 authored by craig[bot]'s avatar craig[bot]

Merge #50379 #50725

50379: opt: modify PushAggDistinctIntoScalarGroupBy to handle GroupBy r=DrewKimball a=DrewKimball

This patch modifies the PushAggDistinctIntoScalarGroupBy rule to
match GroupBy operators.

Release note: None

50725: colexec: materialize whole vectors at once r=yuzefovich a=yuzefovich

**colexec: improve materializer unit tests**

This commit introduces a separate benchmark for the materializer as well
as improves the columnarize-materialize test to use random types. The
latter change allows us to remove another unit test because it became
redundant.

Release note: None

**colexec: materialize whole vectors at once**

Previously, the materializer would perform the conversion one row at
a time, one column at a time. This commit refactors the conversion
method so that it converts the desired number of elements at once and
introduces a helper that can convert the whole batches. The helper is
now used by the materializer and the default builtin operator.

Release note: None
Co-authored-by: default avatarDrew Kimball <[email protected]>
Co-authored-by: default avatarYahor Yuzefovich <[email protected]>
......@@ -897,6 +897,7 @@ EXECGEN_TARGETS = \
pkg/sql/colexec/substring.eg.go \
pkg/sql/colexec/values_differ.eg.go \
pkg/sql/colexec/vec_comparators.eg.go \
pkg/sql/colexec/vec_to_datum.eg.go \
pkg/sql/colexec/window_peer_grouper.eg.go
OPTGEN_TARGETS = \
......
......@@ -18,23 +18,22 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)
type defaultBuiltinFuncOperator struct {
OneInputNode
allocator *colmem.Allocator
evalCtx *tree.EvalContext
funcExpr *tree.FuncExpr
columnTypes []*types.T
argumentCols []int
outputIdx int
outputType *types.T
converter func(tree.Datum) (interface{}, error)
allocator *colmem.Allocator
evalCtx *tree.EvalContext
funcExpr *tree.FuncExpr
columnTypes []*types.T
argumentCols []int
outputIdx int
outputType *types.T
toDatumConverter *vecToDatumConverter
datumToVecConverter func(tree.Datum) (interface{}, error)
row tree.Datums
da sqlbase.DatumAlloc
}
var _ colexecbase.Operator = &defaultBuiltinFuncOperator{}
......@@ -60,17 +59,14 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch {
b.allocator.PerformOperation(
[]coldata.Vec{output},
func() {
b.toDatumConverter.convertBatch(batch)
for i := 0; i < n; i++ {
rowIdx := i
if sel != nil {
rowIdx = sel[i]
}
hasNulls := false
for j := range b.argumentCols {
col := batch.ColVec(b.argumentCols[j])
b.row[j] = PhysicalTypeColElemToDatum(col, rowIdx, &b.da, b.columnTypes[b.argumentCols[j]])
for j, argumentCol := range b.argumentCols {
// Note that we don't need to apply sel to index i because
// vecToDatumConverter returns a "dense" datum column.
b.row[j] = b.toDatumConverter.getDatumColumn(argumentCol)[i]
hasNulls = hasNulls || b.row[j] == tree.DNull
}
......@@ -88,11 +84,16 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch {
}
}
rowIdx := i
if sel != nil {
rowIdx = sel[i]
}
// Convert the datum into a physical type and write it out.
if res == tree.DNull {
batch.ColVec(b.outputIdx).Nulls().SetNull(rowIdx)
output.Nulls().SetNull(rowIdx)
} else {
converted, err := b.converter(res)
converted, err := b.datumToVecConverter(res)
if err != nil {
colexecerror.InternalError(err)
}
......@@ -127,16 +128,17 @@ func NewBuiltinFunctionOperator(
outputType := funcExpr.ResolvedType()
input = newVectorTypeEnforcer(allocator, input, outputType, outputIdx)
return &defaultBuiltinFuncOperator{
OneInputNode: NewOneInputNode(input),
allocator: allocator,
evalCtx: evalCtx,
funcExpr: funcExpr,
outputIdx: outputIdx,
columnTypes: columnTypes,
outputType: outputType,
converter: GetDatumToPhysicalFn(outputType),
row: make(tree.Datums, len(argumentCols)),
argumentCols: argumentCols,
OneInputNode: NewOneInputNode(input),
allocator: allocator,
evalCtx: evalCtx,
funcExpr: funcExpr,
outputIdx: outputIdx,
columnTypes: columnTypes,
outputType: outputType,
toDatumConverter: newVecToDatumConverter(len(columnTypes), argumentCols),
datumToVecConverter: GetDatumToPhysicalFn(outputType),
row: make(tree.Datums, len(argumentCols)),
argumentCols: argumentCols,
}, nil
}
}
......@@ -186,16 +186,17 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) {
b.Fatal(err)
}
defaultOp := &defaultBuiltinFuncOperator{
OneInputNode: NewOneInputNode(source),
allocator: testAllocator,
evalCtx: tctx,
funcExpr: typedExpr.(*tree.FuncExpr),
outputIdx: outputIdx,
columnTypes: typs,
outputType: types.String,
converter: GetDatumToPhysicalFn(types.String),
row: make(tree.Datums, outputIdx),
argumentCols: inputCols,
OneInputNode: NewOneInputNode(source),
allocator: testAllocator,
evalCtx: tctx,
funcExpr: typedExpr.(*tree.FuncExpr),
outputIdx: outputIdx,
columnTypes: typs,
outputType: types.String,
toDatumConverter: newVecToDatumConverter(len(typs), inputCols),
datumToVecConverter: GetDatumToPhysicalFn(types.String),
row: make(tree.Datums, outputIdx),
argumentCols: inputCols,
}
defaultOp.Init()
......
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package main
import (
"fmt"
"io"
"strings"
"text/template"
"github.com/cockroachdb/cockroach/pkg/col/typeconv"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)
type vecToDatumTmplInfo struct {
// TypeFamily contains the type family this struct is handling, with
// "types." prefix.
TypeFamily string
// Widths contains all of the type widths that this struct is handling.
// Note that the entry with 'anyWidth' width must be last in the slice.
Widths []vecToDatumWidthTmplInfo
}
type vecToDatumWidthTmplInfo struct {
Width int32
VecMethod string
// ConversionTmpl is a "format string" for the conversion template. It has
// the same "signature" as AssignConverted, meaning that it should use
// %[1]s for targetElem
// %[2]s for typedCol
// %[3]s for tupleIdx
// %[4]s for datumAlloc.
ConversionTmpl string
}
// AssignConverted returns a string that performs a conversion of the element
// in typedCol at position tupleIdx and assigns the result to targetElem.
// datumAlloc is the name of *sqlbase.DatumAlloc struct that can be used to
// allocate new datums.
func (i vecToDatumWidthTmplInfo) AssignConverted(
targetElem, typedCol, tupleIdx, datumAlloc string,
) string {
return fmt.Sprintf(i.ConversionTmpl, targetElem, typedCol, tupleIdx, datumAlloc)
}
// Remove unused warning.
var _ = vecToDatumWidthTmplInfo{}.AssignConverted
// conversionTmpls maps the type families to the corresponding "format" strings
// (see comment above for details).
// Note that the strings are formatted this way so that generated code doesn't
// have empty lines.
var conversionTmpls = map[types.Family]string{
types.BoolFamily: `%[1]s = tree.MakeDBool(tree.DBool(%[2]s[%[3]s]))`,
// Note that currently, regardless of the integer's width, we always return
// INT8, so there is a single conversion template for IntFamily.
types.IntFamily: `%[1]s = %[4]s.NewDInt(tree.DInt(%[2]s[%[3]s]))`,
types.FloatFamily: `%[1]s = %[4]s.NewDFloat(tree.DFloat(%[2]s[%[3]s]))`,
types.DecimalFamily: ` d := %[4]s.NewDDecimal(tree.DDecimal{Decimal: %[2]s[%[3]s]})
// Clear the Coeff so that the Set below allocates a new slice for the
// Coeff.abs field.
d.Coeff = big.Int{}
d.Coeff.Set(&%[2]s[%[3]s].Coeff)
%[1]s = d`,
types.DateFamily: `%[1]s = %[4]s.NewDDate(tree.DDate{Date: pgdate.MakeCompatibleDateFromDisk(%[2]s[%[3]s])})`,
types.BytesFamily: `// Note that there is no need for a copy since DBytes uses a string
// as underlying storage, which will perform the copy for us.
%[1]s = %[4]s.NewDBytes(tree.DBytes(%[2]s.Get(%[3]s)))`,
types.OidFamily: `%[1]s = %[4]s.NewDOid(tree.MakeDOid(tree.DInt(%[2]s[%[3]s])))`,
types.UuidFamily: ` // Note that there is no need for a copy because uuid.FromBytes
// will perform a copy.
id, err := uuid.FromBytes(%[2]s.Get(%[3]s))
if err != nil {
colexecerror.InternalError(err)
}
%[1]s = %[4]s.NewDUuid(tree.DUuid{UUID: id})`,
types.TimestampFamily: `%[1]s = %[4]s.NewDTimestamp(tree.DTimestamp{Time: %[2]s[%[3]s]})`,
types.TimestampTZFamily: `%[1]s = %[4]s.NewDTimestampTZ(tree.DTimestampTZ{Time: %[2]s[%[3]s]})`,
types.IntervalFamily: `%[1]s = %[4]s.NewDInterval(tree.DInterval{Duration: %[2]s[%[3]s]})`,
typeconv.DatumVecCanonicalTypeFamily: `%[1]s = %[2]s.Get(%[3]s).(*coldataext.Datum).Datum`,
}
const vecToDatumTmpl = "pkg/sql/colexec/vec_to_datum_tmpl.go"
func genVecToDatum(inputFileContents string, wr io.Writer) error {
r := strings.NewReplacer(
"_HAS_NULLS", "$.HasNulls",
"_HAS_SEL", "$.HasSel",
"_TYPE_FAMILY", "{{.TypeFamily}}",
"_TYPE_WIDTH", typeWidthReplacement,
"_VEC_METHOD", "{{.VecMethod}}",
)
s := r.Replace(inputFileContents)
setTupleIdx := makeFunctionRegex("_SET_TUPLE_IDX", 4)
s = setTupleIdx.ReplaceAllString(s, `{{template "setTupleIdx" buildDict "HasSel" $4}}`)
vecToDatum := makeFunctionRegex("_VEC_TO_DATUM", 7)
s = vecToDatum.ReplaceAllString(s, `{{template "vecToDatum" buildDict "Global" . "HasNulls" $6 "HasSel" $7}}`)
assignConvertedRe := makeFunctionRegex("_ASSIGN_CONVERTED", 4)
s = assignConvertedRe.ReplaceAllString(s, makeTemplateFunctionCall("AssignConverted", 4))
tmpl, err := template.New("vec_to_datum").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(s)
if err != nil {
return err
}
var tmplInfos []vecToDatumTmplInfo
// Note that String family is a special case that is handled separately by
// the template explicitly, so it is omitted from this slice.
optimizedTypeFamilies := []types.Family{
types.BoolFamily, types.IntFamily, types.FloatFamily, types.DecimalFamily,
types.DateFamily, types.BytesFamily, types.OidFamily, types.UuidFamily,
types.TimestampFamily, types.TimestampTZFamily, types.IntervalFamily,
}
for _, typeFamily := range optimizedTypeFamilies {
canonicalTypeFamily := typeconv.TypeFamilyToCanonicalTypeFamily(typeFamily)
tmplInfo := vecToDatumTmplInfo{TypeFamily: "types." + typeFamily.String()}
widths := supportedWidthsByCanonicalTypeFamily[canonicalTypeFamily]
if typeFamily != canonicalTypeFamily {
// We have a type family that is supported via another's physical
// representation (e.g. dates are the same as INT8s), so we
// override the widths to use only the default one.
widths = []int32{anyWidth}
}
for _, width := range widths {
tmplInfo.Widths = append(tmplInfo.Widths, vecToDatumWidthTmplInfo{
Width: width,
VecMethod: toVecMethod(canonicalTypeFamily, width),
ConversionTmpl: conversionTmpls[typeFamily],
})
}
tmplInfos = append(tmplInfos, tmplInfo)
}
// Datum-backed types require special handling.
tmplInfos = append(tmplInfos, vecToDatumTmplInfo{
// This special "type family" value will result in matching all type
// families that haven't been matched explicitly, i.e a code like this
// will get generated:
// switch typ.Family() {
// case <all types that have optimized physical representation>
// ...
// case typeconv.DatumVecCanonicalTypeFamily:
// default:
// <datum-vec conversion>
// }
// Such structure requires that datum-vec tmpl info is added last.
TypeFamily: "typeconv.DatumVecCanonicalTypeFamily: default",
Widths: []vecToDatumWidthTmplInfo{{
Width: anyWidth,
VecMethod: toVecMethod(typeconv.DatumVecCanonicalTypeFamily, anyWidth),
ConversionTmpl: conversionTmpls[typeconv.DatumVecCanonicalTypeFamily],
}},
})
return tmpl.Execute(wr, tmplInfos)
}
func init() {
registerGenerator(genVecToDatum, "vec_to_datum.eg.go", vecToDatumTmpl)
}
......@@ -32,21 +32,19 @@ type Materializer struct {
input colexecbase.Operator
typs []*types.T
da sqlbase.DatumAlloc
drainHelper *drainHelper
// runtime fields --
// curIdx represents the current index into the column batch: the next row the
// Materializer will emit.
// curIdx represents the current index into the column batch: the next row
// the Materializer will emit.
curIdx int
// batch is the current Batch the Materializer is processing.
batch coldata.Batch
// colvecs is the unwrapped batch.
colvecs []coldata.Vec
// sel is the selection vector on the batch.
sel []int
// converter contains the converted vectors of the current batch. Note that
// if the batch had a selection vector on top of it, the converted vectors
// will be "dense" and contain only tuples that were selected.
converter *vecToDatumConverter
// row is the memory used for the output row.
row sqlbase.EncDatumRow
......@@ -153,8 +151,10 @@ func NewMaterializer(
input: input,
typs: typs,
drainHelper: newDrainHelper(metadataSourcesQueue),
row: make(sqlbase.EncDatumRow, len(typs)),
closers: toClose,
// nil vecIdxsToConvert indicates that we want to convert all vectors.
converter: newVecToDatumConverter(len(typs), nil /* vecIdxsToConvert */),
row: make(sqlbase.EncDatumRow, len(typs)),
closers: toClose,
}
if err := m.ProcessorBase.Init(
......@@ -220,24 +220,21 @@ func (m *Materializer) next() (sqlbase.EncDatumRow, *execinfrapb.ProducerMetadat
if m.batch == nil || m.curIdx >= m.batch.Length() {
// Get a fresh batch.
m.batch = m.input.Next(m.Ctx)
if m.batch.Length() == 0 {
m.MoveToDraining(nil /* err */)
return nil, m.DrainHelper()
}
m.curIdx = 0
m.colvecs = m.batch.ColVecs()
m.sel = m.batch.Selection()
}
rowIdx := m.curIdx
if m.sel != nil {
rowIdx = m.sel[m.curIdx]
m.converter.convertBatch(m.batch)
}
m.curIdx++
for colIdx, typ := range m.typs {
m.row[colIdx].Datum = PhysicalTypeColElemToDatum(m.colvecs[colIdx], rowIdx, &m.da, typ)
for colIdx := range m.typs {
// Note that we don't need to apply the selection vector of the
// batch to index m.curIdx because vecToDatumConverter returns a
// "dense" datum column.
m.row[colIdx].Datum = m.converter.getDatumColumn(colIdx)[m.curIdx]
}
m.curIdx++
// Note that there is no post-processing to be done in the
// materializer, so we do not use ProcessRowHelper and emit the row
// directly.
......
......@@ -12,27 +12,32 @@ package colexec
import (
"context"
"fmt"
"testing"
"unsafe"
"github.com/cockroachdb/apd/v2"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coldatatestutils"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)
func TestColumnarizeMaterialize(t *testing.T) {
defer leaktest.AfterTest(t)()
// TODO(jordan,asubiotto): add randomness to this test as more types are supported.
typs := []*types.T{types.Int, types.Int}
rng, _ := randutil.NewPseudoRand()
nCols := 1 + rng.Intn(4)
var typs []*types.T
for len(typs) < nCols {
typs = append(typs, sqlbase.RandType(rng))
}
nRows := 10000
nCols := 2
rows := sqlbase.MakeIntRows(nRows, nCols)
rows := sqlbase.RandEncDatumRowsOfTypes(rng, nRows, typs)
input := execinfra.NewRepeatableRowSource(typs, rows)
ctx := context.Background()
......@@ -72,7 +77,7 @@ func TestColumnarizeMaterialize(t *testing.T) {
if row == nil {
t.Fatal("unexpected nil row")
}
for j := 0; j < nCols; j++ {
for j := range typs {
if row[j].Datum.Compare(&evalCtx, rows[i][j].Datum) != 0 {
t.Fatal("unequal rows", row, rows[i])
}
......@@ -87,35 +92,7 @@ func TestColumnarizeMaterialize(t *testing.T) {
}
}
func TestMaterializeTypes(t *testing.T) {
defer leaktest.AfterTest(t)()
// TODO(andyk): Make sure to add more types here. Consider iterating over
// types.OidToTypes list and also using randomly generated EncDatums.
typs := []*types.T{
types.Bool,
types.Int,
types.Float,
types.Decimal,
types.Date,
types.String,
types.Bytes,
types.Name,
types.Oid,
}
inputRow := sqlbase.EncDatumRow{
sqlbase.EncDatum{Datum: tree.DBoolTrue},
sqlbase.EncDatum{Datum: tree.NewDInt(tree.DInt(31))},
sqlbase.EncDatum{Datum: tree.NewDFloat(37.41)},
sqlbase.EncDatum{Datum: &tree.DDecimal{Decimal: *apd.New(43, 47)}},
sqlbase.EncDatum{Datum: tree.NewDDate(pgdate.MakeCompatibleDateFromDisk(53))},
sqlbase.EncDatum{Datum: tree.NewDString("hello")},
sqlbase.EncDatum{Datum: tree.NewDBytes("ciao")},
sqlbase.EncDatum{Datum: tree.NewDName("aloha")},
sqlbase.EncDatum{Datum: tree.NewDOid(59)},
}
input := execinfra.NewRepeatableRowSource(typs, sqlbase.EncDatumRows{inputRow})
func BenchmarkMaterializer(b *testing.B) {
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
evalCtx := tree.MakeTestingEvalContext(st)
......@@ -124,43 +101,76 @@ func TestMaterializeTypes(t *testing.T) {
Cfg: &execinfra.ServerConfig{Settings: st},
EvalCtx: &evalCtx,
}
c, err := NewColumnarizer(ctx, testAllocator, flowCtx, 0, input)
if err != nil {
t.Fatal(err)
}
outputToInputColIdx := make([]int, len(typs))
for i := range outputToInputColIdx {
outputToInputColIdx[i] = i
}
m, err := NewMaterializer(
flowCtx,
1, /* processorID */
c,
typs,
nil, /* output */
nil, /* metadataSourcesQueue */
nil, /* toClose */
nil, /* outputStatsToTrace */
nil, /* cancelFlow */
)
if err != nil {
t.Fatal(err)
}
m.Start(ctx)
row, meta := m.Next()
if meta != nil {
t.Fatalf("unexpected meta %+v", meta)
}
if row == nil {
t.Fatal("unexpected nil row")
}
for i := range inputRow {
inDatum := inputRow[i].Datum
outDatum := row[i].Datum
if inDatum.Compare(&evalCtx, outDatum) != 0 {
t.Fatal("unequal datums", inDatum, outDatum)
rng, _ := randutil.NewPseudoRand()
nBatches := 10
nRows := nBatches * coldata.BatchSize()
for _, typ := range []*types.T{types.Int, types.Float, types.Bytes} {
typs := []*types.T{typ}
nCols := len(typs)
for _, hasNulls := range []bool{false, true} {
for _, useSelectionVector := range []bool{false, true} {
b.Run(fmt.Sprintf("%s/hasNulls=%t/useSel=%t", typ, hasNulls, useSelectionVector), func(b *testing.B) {
nullProb := 0.0
if hasNulls {
nullProb = nullProbability
}
batch := testAllocator.NewMemBatch(typs)
for _, colVec := range batch.ColVecs() {
coldatatestutils.RandomVec(coldatatestutils.RandomVecArgs{
Rand: rng,
Vec: colVec,
N: coldata.BatchSize(),
NullProbability: nullProb,
BytesFixedLength: 8,
})
}
batch.SetLength(coldata.BatchSize())
if useSelectionVector {
batch.SetSelection(true)
sel := batch.Selection()
for i := 0; i < coldata.BatchSize(); i++ {
sel[i] = i
}
}
input := newFiniteBatchSource(batch, typs, nBatches)
b.SetBytes(int64(nRows * nCols * int(unsafe.Sizeof(int64(0)))))
for i := 0; i < b.N; i++ {
m, err := NewMaterializer(
flowCtx,
0, /* processorID */
input,
typs,
nil, /* output */
nil, /* metadataSourcesQueue */
nil, /* toClose */
nil, /* outputStatsToTrace */
nil, /* cancelFlow */
)
if err != nil {
b.Fatal(err)
}
m.Start(ctx)
foundRows := 0
for {
row, meta := m.Next()
if meta != nil {
b.Fatalf("unexpected metadata %v", meta)
}
if row == nil {
break
}
foundRows++
}
if foundRows != nRows {
b.Fatalf("expected %d rows, found %d", nRows, foundRows)
}
input.reset(nBatches)
}
})
}
}
}
}
......
......@@ -261,10 +261,8 @@ func TestRandomComparisons(t *testing.T) {
},
)
}
for i := range lDatums {
lDatums[i] = PhysicalTypeColElemToDatum(lVec, i, &da, typ)
rDatums[i] = PhysicalTypeColElemToDatum(rVec, i, &da, typ)
}
PhysicalTypeColVecToDatum(lDatums, lVec, numTuples, nil /* sel */, &da)
PhysicalTypeColVecToDatum(rDatums, rVec, numTuples, nil /* sel */, &da)
supportedCmpOps := []tree.ComparisonOperator{tree.EQ, tree.NE, tree.LT, tree.LE, tree.GT, tree.GE}
if typ.Family() == types.JsonFamily {
supportedCmpOps = []tree.ComparisonOperator{tree.EQ, tree.NE}
......
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package colexec
import (
"math/big"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coldataext"
"github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/lib/pq/oid"
)
// PhysicalTypeColElemToDatum converts an element in a colvec to a datum of
// type ct. The returned Datum is a deep copy of the colvec element. Note
// that this function handles nulls as well, so there is no need for a separate
// null check.
func PhysicalTypeColElemToDatum(
col coldata.Vec, rowIdx int, da *sqlbase.DatumAlloc, ct *types.T,
) tree.Datum {
if col.MaybeHasNulls() {
if col.Nulls().NullAt(rowIdx) {
return tree.DNull
}
}
switch ct.Family() {
case types.BoolFamily:
if col.Bool()[rowIdx] {
return tree.DBoolTrue
}
return tree.DBoolFalse
case types.IntFamily:
switch ct.Width() {
case 16:
return da.NewDInt(tree.DInt(col.Int16()[rowIdx]))
case 32:
return da.NewDInt(tree.DInt(col.Int32()[rowIdx]))
default:
return da.NewDInt(tree.DInt(col.Int64()[rowIdx]))
}
case types.FloatFamily:
return da.NewDFloat(tree.DFloat(col.Float64()[rowIdx]))
case types.DecimalFamily:
d := da.NewDDecimal(tree.DDecimal{Decimal: col.Decimal()[rowIdx]})
// Clear the Coeff so that the Set below allocates a new slice for the
// Coeff.abs field.
d.Coeff = big.Int{}
d.Coeff.Set(&col.Decimal()[rowIdx].Coeff)
return d
case types.DateFamily:
return tree.NewDDate(pgdate.MakeCompatibleDateFromDisk(col.Int64()[rowIdx]))
case types.StringFamily:
// Note that there is no need for a copy since casting to a string will do
// that.
b := col.Bytes().Get(rowIdx)
if ct.Oid() == oid.T_name {
return da.NewDName(tree.DString(string(b)))
}
return da.NewDString(tree.DString(string(b)))
case types.BytesFamily:
// Note that there is no need for a copy since DBytes uses a string as
// underlying storage, which will perform the copy for us.
return da.NewDBytes(tree.DBytes(col.Bytes().Get(rowIdx)))
case types.OidFamily:
return da.NewDOid(tree.MakeDOid(tree.DInt(col.Int64()[rowIdx])))
case types.UuidFamily:
// Note that there is no need for a copy because uuid.FromBytes will perform
// a copy.
id, err := uuid.FromBytes(col.Bytes().Get(rowIdx))
if err != nil {
colexecerror.InternalError(err)
}
return da.NewDUuid(tree.DUuid{UUID: id})
case types.TimestampFamily:
return da.NewDTimestamp(tree.DTimestamp{Time: col.Timestamp()[rowIdx]})
case types.TimestampTZFamily:
return da.NewDTimestampTZ(tree.DTimestampTZ{Time: col.Timestamp()[rowIdx]})
case types.IntervalFamily:
return da.NewDInterval(tree.DInterval{Duration: col.Interval()[rowIdx]})
default:
return col.Datum().Get(rowIdx).(*coldataext.Datum).Datum