Skip to content

Commit

Permalink
[DNM] sql: combine DDecimal alloc with big.Int coefficient alloc
Browse files Browse the repository at this point in the history
Similar to the previous commit, but for non-vectorized execution.

Needs polish.
  • Loading branch information
nvanbenschoten committed Jan 1, 2022
1 parent f22bb23 commit 9512cb4
Show file tree
Hide file tree
Showing 17 changed files with 114 additions and 82 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_mysql.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,7 @@ func mysqlValueToDatum(
return tree.NewDFloat(-*i), nil
case *tree.DDecimal:
dec := &i.Decimal
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
dd.Decimal.Neg(dec)
return dd, nil
default:
Expand Down
10 changes: 5 additions & 5 deletions pkg/ccl/importccl/read_import_workload.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"sync/atomic"
"unsafe"

"github.com/cockroachdb/apd/v2"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -73,8 +72,9 @@ func makeDatumFromColOffset(
case types.IntFamily:
return alloc.NewDInt(tree.DInt(col.Int64()[rowIdx])), nil
case types.DecimalFamily:
d := *apd.New(col.Int64()[rowIdx], 0)
return alloc.NewDDecimal(tree.DDecimal{Decimal: d}), nil
d := alloc.NewDDecimal(tree.DDecimal{})
d.SetInt64(col.Int64()[rowIdx])
return d, nil
case types.DateFamily:
date, err := pgdate.MakeDateFromUnixEpoch(col.Int64()[rowIdx])
if err != nil {
Expand All @@ -93,11 +93,11 @@ func makeDatumFromColOffset(
case types.FloatFamily:
return alloc.NewDFloat(tree.DFloat(col.Float64()[rowIdx])), nil
case types.DecimalFamily:
var d apd.Decimal
d := alloc.NewDDecimal(tree.DDecimal{})
if _, err := d.SetFloat64(col.Float64()[rowIdx]); err != nil {
return nil, err
}
return alloc.NewDDecimal(tree.DDecimal{Decimal: d}), nil
return d, nil
}
case types.BytesFamily:
switch hint.Family() {
Expand Down
11 changes: 9 additions & 2 deletions pkg/sql/colexec/colexectestutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -1401,8 +1401,15 @@ func tupleEquals(expected Tuple, actual Tuple, evalCtx *tree.EvalContext) bool {
}
// Special case for decimals.
if d1, ok := actual[i].(apd.Decimal); ok {
if f2, ok := expected[i].(float64); ok {
d2, _, err := apd.NewFromString(fmt.Sprintf("%f", f2))
switch t := expected[i].(type) {
case apd.Decimal:
if d1.Cmp(&t) == 0 {
continue
} else {
return false
}
case float64:
d2, _, err := apd.NewFromString(fmt.Sprintf("%f", t))
if err == nil && d1.Cmp(d2) == 0 {
continue
} else {
Expand Down
12 changes: 4 additions & 8 deletions pkg/sql/colexec/execgen/cmd/execgen/vec_to_datum_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,14 +67,10 @@ var vecToDatumConversionTmpls = map[types.Family]string{
types.BoolFamily: `%[1]s := tree.MakeDBool(tree.DBool(%[2]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 := %[3]s.NewDInt(tree.DInt(%[2]s))`,
types.FloatFamily: `%[1]s := %[3]s.NewDFloat(tree.DFloat(%[2]s))`,
types.DecimalFamily: ` %[1]s := %[3]s.NewDDecimal(tree.DDecimal{Decimal: %[2]s})
// Clear the Coeff so that the Set below allocates a new slice for the
// Coeff.abs field.
%[1]s.Coeff = big.Int{}
%[1]s.Coeff.Set(&%[2]s.Coeff)`,
types.DateFamily: `%[1]s := %[3]s.NewDDate(tree.DDate{Date: pgdate.MakeCompatibleDateFromDisk(%[2]s)})`,
types.IntFamily: `%[1]s := %[3]s.NewDInt(tree.DInt(%[2]s))`,
types.FloatFamily: `%[1]s := %[3]s.NewDFloat(tree.DFloat(%[2]s))`,
types.DecimalFamily: `%[1]s := %[3]s.NewDDecimal(tree.DDecimal{Decimal: %[2]s})`,
types.DateFamily: `%[1]s := %[3]s.NewDDate(tree.DDate{Date: pgdate.MakeCompatibleDateFromDisk(%[2]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 := %[3]s.NewDBytes(tree.DBytes(%[2]s))`,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1554,7 +1554,7 @@ func golangFillQueryArguments(args ...interface{}) (tree.Datums, error) {
case bitarray.BitArray:
d = &tree.DBitArray{BitArray: t}
case *apd.Decimal:
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
dd.Set(t)
d = dd
case security.SQLUsername:
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/randgen/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func RandDatumWithNullChance(rng *rand.Rand, typ *types.T, nullChance int) tree.
}
return tree.NewDGeometry(geogen.RandomGeometry(rng, gm.SRID))
case types.DecimalFamily:
d := &tree.DDecimal{}
d := tree.NewDDecimal()
// int64(rng.Uint64()) to get negative numbers, too
d.Decimal.SetFinite(int64(rng.Uint64()), int32(rng.Intn(40)-20))
return d
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/rowenc/datum_alloc.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ type DatumAlloc struct {
dstringAlloc []tree.DString
dbytesAlloc []tree.DBytes
dbitArrayAlloc []tree.DBitArray
ddecimalAlloc []tree.DDecimal
ddecimalAlloc []tree.DDecimalAlloc
ddateAlloc []tree.DDate
denumAlloc []tree.DEnum
dbox2dAlloc []tree.DBox2D
Expand Down Expand Up @@ -170,10 +170,10 @@ func (a *DatumAlloc) NewDDecimal(v tree.DDecimal) *tree.DDecimal {
}
buf := &a.ddecimalAlloc
if len(*buf) == 0 {
*buf = make([]tree.DDecimal, a.AllocSize)
*buf = make([]tree.DDecimalAlloc, a.AllocSize)
}
r := &(*buf)[0]
*r = v
r := (*buf)[0].Get()
r.Set(&v.Decimal)
*buf = (*buf)[1:]
return r
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/rowexec/processor_utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ func toEncDatum(datumType *types.T, v interface{}) rowenc.EncDatum {
switch concreteType := v.(type) {
case int:
if datumType.Family() == types.DecimalFamily {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
dd.SetInt64(int64(v.(int)))
return dd
}
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/sem/builtins/aggregate_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -2577,7 +2577,7 @@ func (a *intSumAggregate) Result() (tree.Datum, error) {
if !a.seenNonNull {
return tree.DNull, nil
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
if a.large {
dd.Set(&a.decSum)
} else {
Expand Down Expand Up @@ -2644,7 +2644,7 @@ func (a *decimalSumAggregate) Result() (tree.Datum, error) {
if !a.sawNonNull {
return tree.DNull, nil
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
dd.Set(&a.sum)
return dd, nil
}
Expand Down Expand Up @@ -2947,7 +2947,7 @@ func (a *decimalSqrDiffAggregate) Result() (tree.Datum, error) {
if a.count.Cmp(decimalOne) < 0 {
return tree.DNull, nil
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
dd.Set(&a.sqrDiff)
// Remove trailing zeros. Depending on the order in which the input
// is processed, some number of trailing zeros could be added to the
Expand Down Expand Up @@ -3267,7 +3267,7 @@ func (a *decimalVarianceAggregate) Result() (tree.Datum, error) {
if _, err = tree.IntermediateCtx.Sub(a.agg.Tmp(), a.agg.Count(), decimalOne); err != nil {
return nil, err
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
if _, err = tree.DecimalCtx.Quo(&dd.Decimal, &sqrDiff.(*tree.DDecimal).Decimal, a.agg.Tmp()); err != nil {
return nil, err
}
Expand Down Expand Up @@ -3382,7 +3382,7 @@ func (a *decimalVarPopAggregate) Result() (tree.Datum, error) {
if err != nil {
return nil, err
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
if _, err = tree.DecimalCtx.Quo(&dd.Decimal, &sqrDiff.(*tree.DDecimal).Decimal, a.agg.Count()); err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sem/builtins/aggregate_builtins_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -499,7 +499,7 @@ func makeDecimalTestDatum(count int) []tree.Datum {

vals := make([]tree.Datum, count)
for i := range vals {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
if _, err := dd.SetFloat64(rng.Float64()); err != nil {
panic(err)
}
Expand Down
24 changes: 12 additions & 12 deletions pkg/sql/sem/builtins/math_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ var mathBuiltins = map[string]builtinDefinition{
return tree.NewDFloat(tree.DFloat(math.Abs(x))), nil
}, "Calculates the absolute value of `val`.", tree.VolatilityImmutable),
decimalOverload1(func(x *apd.Decimal) (tree.Datum, error) {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
dd.Abs(x)
return dd, nil
}, "Calculates the absolute value of `val`.", tree.VolatilityImmutable),
Expand Down Expand Up @@ -197,7 +197,7 @@ var mathBuiltins = map[string]builtinDefinition{
if y.Sign() == 0 {
return nil, tree.ErrDivByZero
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.HighPrecisionCtx.QuoInteger(&dd.Decimal, x, y)
return dd, err
}, "Calculates the integer quotient of `x`/`y`.", tree.VolatilityImmutable),
Expand All @@ -222,7 +222,7 @@ var mathBuiltins = map[string]builtinDefinition{
return tree.NewDFloat(tree.DFloat(math.Exp(x))), nil
}, "Calculates *e* ^ `val`.", tree.VolatilityImmutable),
decimalOverload1(func(x *apd.Decimal) (tree.Datum, error) {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.DecimalCtx.Exp(&dd.Decimal, x)
return dd, err
}, "Calculates *e* ^ `val`.", tree.VolatilityImmutable),
Expand All @@ -233,7 +233,7 @@ var mathBuiltins = map[string]builtinDefinition{
return tree.NewDFloat(tree.DFloat(math.Floor(x))), nil
}, "Calculates the largest integer not greater than `val`.", tree.VolatilityImmutable),
decimalOverload1(func(x *apd.Decimal) (tree.Datum, error) {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.ExactCtx.Floor(&dd.Decimal, x)
return dd, err
}, "Calculates the largest integer not greater than `val`.", tree.VolatilityImmutable),
Expand Down Expand Up @@ -322,7 +322,7 @@ var mathBuiltins = map[string]builtinDefinition{
return nil, err
}

dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.DecimalCtx.Quo(&dd.Decimal, top, bot)
return dd, err
}, "Calculates the base `b` log of `val`.", tree.VolatilityImmutable),
Expand All @@ -336,7 +336,7 @@ var mathBuiltins = map[string]builtinDefinition{
if y.Sign() == 0 {
return nil, tree.ErrDivByZero
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.HighPrecisionCtx.Rem(&dd.Decimal, x, y)
return dd, err
}, "Calculates `x`%`y`.", tree.VolatilityImmutable),
Expand Down Expand Up @@ -462,7 +462,7 @@ var mathBuiltins = map[string]builtinDefinition{
}, "Determines the sign of `val`: **1** for positive; **0** for 0 values; **-1** for "+
"negative.", tree.VolatilityImmutable),
decimalOverload1(func(x *apd.Decimal) (tree.Datum, error) {
d := &tree.DDecimal{}
d := tree.NewDDecimal()
d.Decimal.SetInt64(int64(x.Sign()))
return d, nil
}, "Determines the sign of `val`: **1** for positive; **0** for 0 values; **-1** for "+
Expand Down Expand Up @@ -518,7 +518,7 @@ var mathBuiltins = map[string]builtinDefinition{
return tree.NewDFloat(tree.DFloat(math.Trunc(x))), nil
}, "Truncates the decimal values of `val`.", tree.VolatilityImmutable),
decimalOverload1(func(x *apd.Decimal) (tree.Datum, error) {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
x.Modf(&dd.Decimal, nil)
return dd, nil
}, "Truncates the decimal values of `val`.", tree.VolatilityImmutable),
Expand Down Expand Up @@ -593,7 +593,7 @@ var ceilImpl = makeBuiltin(defProps(),
return tree.NewDFloat(tree.DFloat(math.Ceil(x))), nil
}, "Calculates the smallest integer not smaller than `val`.", tree.VolatilityImmutable),
decimalOverload1(func(x *apd.Decimal) (tree.Datum, error) {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.ExactCtx.Ceil(&dd.Decimal, x)
if dd.IsZero() {
dd.Negative = false
Expand All @@ -616,7 +616,7 @@ var powImpls = makeBuiltin(defProps(),
return tree.NewDFloat(tree.DFloat(math.Pow(x, y))), nil
}, "Calculates `x`^`y`.", tree.VolatilityImmutable),
decimalOverload2("x", "y", func(x, y *apd.Decimal) (tree.Datum, error) {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.DecimalCtx.Pow(&dd.Decimal, x, y)
return dd, err
}, "Calculates `x`^`y`.", tree.VolatilityImmutable),
Expand Down Expand Up @@ -646,7 +646,7 @@ func decimalLogFn(
case 0:
return nil, errLogOfZero
}
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := logFn(&dd.Decimal, x)
return dd, err
}, info, tree.VolatilityImmutable)
Expand Down Expand Up @@ -730,7 +730,7 @@ func roundDDecimal(d *tree.DDecimal, scale int32) (tree.Datum, error) {
}

func roundDecimal(x *apd.Decimal, scale int32) (tree.Datum, error) {
dd := &tree.DDecimal{}
dd := tree.NewDDecimal()
_, err := tree.HighPrecisionCtx.Quantize(&dd.Decimal, x, -scale)
return dd, err
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/sem/builtins/window_frame_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,9 +246,9 @@ func (w *slidingWindowSumFunc) removeAllBefore(
case *tree.DInt:
err = w.agg.Add(ctx, tree.NewDInt(-*v))
case *tree.DDecimal:
d := tree.DDecimal{}
d := tree.NewDDecimal()
d.Neg(&v.Decimal)
err = w.agg.Add(ctx, &d)
err = w.agg.Add(ctx, d)
case *tree.DFloat:
err = w.agg.Add(ctx, tree.NewDFloat(-*v))
case *tree.DInterval:
Expand Down Expand Up @@ -405,7 +405,7 @@ func (w *avgWindowFunc) Compute(
_, err := tree.DecimalCtx.Quo(&avg.Decimal, &t.Decimal, count)
return &avg, err
case *tree.DInt:
dd := tree.DDecimal{}
dd := tree.NewDDecimal()
dd.SetInt64(int64(*t))
var avg tree.DDecimal
count := apd.New(int64(frameSize), 0)
Expand Down
31 changes: 30 additions & 1 deletion pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -996,6 +996,35 @@ type DDecimal struct {
apd.Decimal
}

// DDecimalAlloc attempts to minimize heap allocations when using a DDecimal.
// Each Decimal maintains (through an embedded big.Int) an internal reference
// to a variable-length coefficient, which is represented by a []big.Word. To
// minimize the number of individual heap allocations, we allocate a small
// []big.Word alongside the DDecimal and wire the two up. big.Int will avoid
// re-allocating unless its coefficient exceeds the initial capacity. We set
// this capacity to accommodate any coefficient that would fit in a 64-bit
// integer.
type DDecimalAlloc struct {
d DDecimal
w [unsafe.Sizeof(uint64(0)) / unsafe.Sizeof(big.Word(0))]big.Word
}

// Get returns the allocated DDecimal.
func (d *DDecimalAlloc) Get() *DDecimal {
if d.d.Coeff.Bits() == nil {
d.d.Coeff.SetBits(d.w[:])
}
return &d.d
}

// NewDDecimal is a helper routine to create a *DDecimal. The function attempts
// to allocate the Decimal and its internal, variable-length references in a
// single heap allocation.
func NewDDecimal() *DDecimal {
a := new(DDecimalAlloc)
return a.Get()
}

// MustBeDDecimal attempts to retrieve a DDecimal from an Expr, panicking if the
// assertion fails.
func MustBeDDecimal(e Expr) DDecimal {
Expand All @@ -1009,7 +1038,7 @@ func MustBeDDecimal(e Expr) DDecimal {
// ParseDDecimal parses and returns the *DDecimal Datum value represented by the
// provided string, or an error if parsing is unsuccessful.
func ParseDDecimal(s string) (*DDecimal, error) {
dd := &DDecimal{}
dd := NewDDecimal()
err := dd.SetString(s)
return dd, err
}
Expand Down
Loading

0 comments on commit 9512cb4

Please sign in to comment.