Skip to content

Commit

Permalink
sem/tree: add support for producing vectorized data from strings
Browse files Browse the repository at this point in the history
tree.ValueHandler exposes raw machine type hooks that are used by
vec_handler to build coldata.Vec's.

Epic: CRDB-18892
Informs: #91831
Release note: None
  • Loading branch information
cucaroach committed Feb 15, 2023
1 parent f91cf78 commit 7cdc27a
Show file tree
Hide file tree
Showing 6 changed files with 552 additions and 13 deletions.
6 changes: 6 additions & 0 deletions pkg/col/coldataext/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
srcs = [
"datum_vec.go",
"extended_column_factory.go",
"vec_handler.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/col/coldataext",
visibility = ["//visibility:public"],
Expand All @@ -20,6 +21,11 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util/buildutil",
"//pkg/util/duration",
"//pkg/util/encoding",
"//pkg/util/json",
"//pkg/util/timeutil/pgdate",
"@com_github_cockroachdb_apd_v3//:apd",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand Down
159 changes: 159 additions & 0 deletions pkg/col/coldataext/vec_handler.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,159 @@
// Copyright 2023 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 coldataext

import (
"time"

"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/typeconv"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/json"
"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/errors"
)

// MakeVecHandler makes a tree.ValueHandler that stores values to a coldata.Vec.
func MakeVecHandler(vec coldata.Vec) tree.ValueHandler {
v := vecHandler{nulls: vec.Nulls()}
switch vec.CanonicalTypeFamily() {
case types.BoolFamily:
v.bools = vec.Bool()
case types.BytesFamily:
v.bytes = vec.Bytes()
case types.DecimalFamily:
v.decimals = vec.Decimal()
case types.IntFamily:
v.ints = vec.Int64()
case types.FloatFamily:
v.floats = vec.Float64()
case types.TimestampTZFamily:
v.timestamps = vec.Timestamp()
case types.IntervalFamily:
v.intervals = vec.Interval()
case types.JsonFamily:
v.jsons = vec.JSON()
case typeconv.DatumVecCanonicalTypeFamily:
v.datums = vec.Datum()
default:
colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", vec.Type()))
}
return &v
}

type vecHandler struct {
nulls *coldata.Nulls
bools coldata.Bools
bytes *coldata.Bytes
decimals coldata.Decimals
// TODO(cucaroach): implement small int types
//int16s coldata.Int16s
//int32s coldata.Int32s
ints coldata.Int64s
floats coldata.Float64s
timestamps coldata.Times
intervals coldata.Durations
jsons *coldata.JSONs
datums coldata.DatumVec
row int
}

var _ tree.ValueHandler = (*vecHandler)(nil)

// Reset is used to re-use a batch handler across batches.
func (v *vecHandler) Reset() {
v.row = 0
}

// Len returns the current length of the vector.
func (v *vecHandler) Len() int {
return v.row
}

// Decimal implements tree.ValueHandler interface. It returns a pointer into the
// vec to allow the decimal to be constructed in place which avoids expensive
// copying and temporary allocations.
func (v *vecHandler) Decimal() *apd.Decimal {
d := &v.decimals[v.row]
v.row++
return d
}

// Null implements tree.ValueHandler interface.
func (v *vecHandler) Null() {
v.nulls.SetNull(v.row)
v.row++
}

// String is part of the tree.ValueHandler interface.
func (v *vecHandler) String(s string) {
v.bytes.Set(v.row, encoding.UnsafeConvertStringToBytes(s))
v.row++
}

// Date is part of the tree.ValueHandler interface.
func (v *vecHandler) Date(d pgdate.Date) {
v.ints[v.row] = d.UnixEpochDaysWithOrig()
v.row++
}

// Datum is part of the tree.ValueHandler interface.
func (v *vecHandler) Datum(d tree.Datum) {
v.datums.Set(v.row, d)
v.row++
}

// Bool is part of the tree.ValueHandler interface.
func (v *vecHandler) Bool(b bool) {
v.bools[v.row] = b
v.row++
}

// Bytes is part of the tree.ValueHandler interface.
func (v *vecHandler) Bytes(b []byte) {
v.bytes.Set(v.row, b)
v.row++
}

// Float is part of the tree.ValueHandler interface.
func (v *vecHandler) Float(f float64) {
v.floats[v.row] = f
v.row++
}

// Int is part of the tree.ValueHandler interface.
func (v *vecHandler) Int(i int64) {
v.ints[v.row] = i
v.row++
}

// Duration is part of the tree.ValueHandler interface.
func (v *vecHandler) Duration(d duration.Duration) {
v.intervals[v.row] = d
v.row++
}

// JSON is part of the tree.ValueHandler interface.
func (v *vecHandler) JSON(j json.JSON) {
v.jsons.Set(v.row, j)
v.row++
}

// TimestampTZ is part of the tree.ValueHandler interface.
func (v *vecHandler) TimestampTZ(t time.Time) {
v.timestamps[v.row] = t
v.row++
}
5 changes: 5 additions & 0 deletions pkg/sql/sem/tree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree",
visibility = ["//visibility:public"],
deps = [
"//pkg/col/typeconv", # keep
"//pkg/geo",
"//pkg/geo/geopb",
"//pkg/sql/lex",
Expand Down Expand Up @@ -193,6 +194,7 @@ go_test(
"operators_test.go",
"overload_test.go",
"parse_array_test.go",
"parse_string_test.go",
"parse_tuple_test.go",
"placeholders_test.go",
"pretty_test.go",
Expand All @@ -210,10 +212,13 @@ go_test(
embed = [":tree"],
deps = [
"//pkg/build/bazel",
"//pkg/col/coldata",
"//pkg/col/coldataext",
"//pkg/internal/rsg",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/settings/cluster",
"//pkg/sql/colconv",
"//pkg/sql/parser",
"//pkg/sql/randgen",
"//pkg/sql/sem/builtins",
Expand Down
27 changes: 22 additions & 5 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -1049,10 +1049,14 @@ func ParseDDecimal(s string) (*DDecimal, error) {
// SetString sets d to s. Any non-standard NaN values are converted to a
// normal NaN. Any negative zero is converted to positive.
func (d *DDecimal) SetString(s string) error {
return setDecimalString(s, &d.Decimal)
}

func setDecimalString(s string, d *apd.Decimal) error {
// ExactCtx should be able to handle any decimal, but if there is any rounding
// or other inexact conversion, it will result in an error.
// _, res, err := HighPrecisionCtx.SetString(&d.Decimal, s)
_, res, err := ExactCtx.SetString(&d.Decimal, s)
_, res, err := ExactCtx.SetString(d, s)
if res != 0 || err != nil {
return MakeParseError(s, types.Decimal, err)
}
Expand Down Expand Up @@ -2606,6 +2610,9 @@ var DZeroTimestamp = &DTimestamp{}
//
// The dependsOnContext return value indicates if we had to consult the
// ParseContext (either for the time or the local timezone).
//
// Parts of this function are inlined into ParseAndRequireStringHandler, if this
// changes materially the timestamp case arms there may need to change too.
func ParseDTimestamp(
ctx ParseContext, s string, precision time.Duration,
) (_ *DTimestamp, dependsOnContext bool, _ error) {
Expand Down Expand Up @@ -2854,13 +2861,20 @@ type DTimestampTZ struct {
time.Time
}

// MakeDTimestampTZ creates a DTimestampTZ with specified precision.
func MakeDTimestampTZ(t time.Time, precision time.Duration) (*DTimestampTZ, error) {
func checkTimeBounds(t time.Time, precision time.Duration) (time.Time, error) {
ret := t.Round(precision)
if ret.After(MaxSupportedTime) || ret.Before(MinSupportedTime) {
return nil, NewTimestampExceedsBoundsError(ret)
return time.Time{}, NewTimestampExceedsBoundsError(ret)
}
return &DTimestampTZ{Time: ret}, nil
return ret, nil
}

// MakeDTimestampTZ creates a DTimestampTZ with specified precision.
func MakeDTimestampTZ(t time.Time, precision time.Duration) (_ *DTimestampTZ, err error) {
if t, err = checkTimeBounds(t, precision); err != nil {
return nil, err
}
return &DTimestampTZ{Time: t}, nil
}

// MustMakeDTimestampTZ wraps MakeDTimestampTZ but panics if there is an error.
Expand Down Expand Up @@ -2891,6 +2905,9 @@ func MakeDTimestampTZFromDate(loc *time.Location, d *DDate) (*DTimestampTZ, erro
//
// The dependsOnContext return value indicates if we had to consult the
// ParseContext (either for the time or the local timezone).
//
// Parts of this function are inlined into ParseAndRequireStringHandler, if this
// changes materially the timestamp case arms there may need to change too.
func ParseDTimestampTZ(
ctx ParseContext, s string, precision time.Duration,
) (_ *DTimestampTZ, dependsOnContext bool, _ error) {
Expand Down
Loading

0 comments on commit 7cdc27a

Please sign in to comment.