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 Jan 31, 2023
1 parent 58762ac commit 07debe9
Show file tree
Hide file tree
Showing 4 changed files with 293 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
119 changes: 119 additions & 0 deletions pkg/col/coldataext/vec_handler.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
// Copyright 2022 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/sql/sem/tree"
"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"
)

// MakeVecHandler makes a tree.ValueHandler that stores values to a coldata.Vec.
func MakeVecHandler(v coldata.Vec) tree.ValueHandler {
return &vecHandler{col: v}
}

type vecHandler struct {
col coldata.Vec
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 avoid copying.
func (v *vecHandler) Decimal() *apd.Decimal {
d := &v.col.Decimal()[v.row]
v.row++
return d
}

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

// String is part of the tree.ValueHandler interface.
func (v *vecHandler) String(s string) {
v.col.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.col.Int64().Set(v.row, d.UnixEpochDaysWithOrig())
v.row++
}

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

// Bool is part of the tree.ValueHandler interface.
func (v *vecHandler) Bool(b bool) {
v.col.Bool().Set(v.row, b)
v.row++
}

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

// Float is part of the tree.ValueHandler interface.
func (v *vecHandler) Float(f float64) {
v.col.Float64().Set(v.row, f)
v.row++
}

// Int is part of the tree.ValueHandler interface.
func (v *vecHandler) Int(i int64) {
v.col.Int64().Set(v.row, i)
v.row++
}

// Duration is part of the tree.ValueHandler interface.
func (v *vecHandler) Duration(d duration.Duration) {
v.col.Interval().Set(v.row, d)
v.row++
}

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

// TimestampTZ is part of the tree.ValueHandler interface.
func (v *vecHandler) TimestampTZ(t time.Time) {
v.col.Timestamp().Set(v.row, t)
v.row++
}
29 changes: 24 additions & 5 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -1041,10 +1041,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 @@ -2609,6 +2613,10 @@ const (
//
// 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 ParseAndRequireStringToValueHandler,
// 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 @@ -2860,13 +2868,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 @@ -2897,6 +2912,10 @@ 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 ParseAndRequireStringToValueHandler,
// 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
152 changes: 144 additions & 8 deletions pkg/sql/sem/tree/parse_string.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,15 +13,24 @@ package tree
import (
"strconv"
"strings"
"time"

"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"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/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)

// ParseAndRequireString parses s as type t for simple types. Collated
// strings are not handled.
// ParseAndRequireString parses s as type t for simple types.
//
// The dependsOnContext return value indicates if we had to consult the
// ParseContext (either for the time or the local timezone).
Expand Down Expand Up @@ -74,12 +83,7 @@ func ParseAndRequireString(
case types.CollatedStringFamily:
d, err = NewDCollatedString(s, t.Locale(), ctx.GetCollationEnv())
case types.StringFamily:
// If the string type specifies a limit we truncate to that limit:
// 'hello'::CHAR(2) -> 'he'
// This is true of all the string type variants.
if t.Width() > 0 {
s = util.TruncateString(s, int(t.Width()))
}
s = truncateString(s, t)
return NewDString(s), false, nil
case types.TimeFamily:
d, dependsOnContext, err = ParseDTime(ctx, s, TimeFamilyPrecisionToRoundDuration(t.Precision()))
Expand Down Expand Up @@ -115,6 +119,16 @@ func ParseAndRequireString(
return d, dependsOnContext, err
}

func truncateString(s string, t *types.T) string {
// If the string type specifies a limit we truncate to that limit:
// 'hello'::CHAR(2) -> 'he'
// This is true of all the string type variants.
if t.Width() > 0 {
s = util.TruncateString(s, int(t.Width()))
}
return s
}

// ParseDOidAsInt parses the input and returns it as an OID. If the input
// is not formatted as an int, an error is returned.
func ParseDOidAsInt(s string) (*DOid, error) {
Expand Down Expand Up @@ -143,3 +157,125 @@ func FormatBitArrayToType(d *DBitArray, t *types.T) *DBitArray {
}
return &DBitArray{a}
}

// ValueHandler is an interface to allow raw types to be extracted from strings.
type ValueHandler interface {
Len() int
Null()
Date(d pgdate.Date)
Datum(d Datum)
Bool(b bool)
Bytes(b []byte)
// Decimal returns a pointer into the vec for in place construction.
Decimal() *apd.Decimal
Float(f float64)
Int(i int64)
Duration(d duration.Duration)
JSON(j json.JSON)
String(s string)
TimestampTZ(t time.Time)
Reset()
}

// ParseAndRequireStringToValueHandler parses a string and passes values
// supported by the vector engine directly to a ValueHandler. Other types are
// handled by ParseAndRequireString.
func ParseAndRequireStringToValueHandler(
t *types.T, s string, ctx ParseContext, vh ValueHandler, ph *pgdate.ParseHelper,
) (err error) {
switch t.Family() {
case types.BoolFamily:
var b bool
if b, err = ParseBool(strings.TrimSpace(s)); err == nil {
vh.Bool(b)
}
case types.BytesFamily:
var res []byte
if res, err = lex.DecodeRawBytesToByteArrayAuto(encoding.UnsafeConvertStringToBytes(s)); err == nil {
vh.Bytes(res)
} else {
err = MakeParseError(s, types.Bytes, err)
}
case types.DateFamily:
now := relativeParseTime(ctx)
var t pgdate.Date
if t, _, err = pgdate.ParseDate(now, dateStyle(ctx), s, ph); err == nil {
vh.Date(t)
}
case types.DecimalFamily:
// Decimal is a little different to allow in place construction.
dec := vh.Decimal()
if err = setDecimalString(s, dec); err != nil {
// Erase any invalid results.
*dec = apd.Decimal{}
}
case types.FloatFamily:
var f float64
if f, err = strconv.ParseFloat(s, 64); err == nil {
vh.Float(f)
} else {
err = MakeParseError(s, types.Float, err)
}
case types.IntFamily:
var i int64
if i, err = strconv.ParseInt(s, 0, 64); err == nil {
vh.Int(i)
} else {
err = MakeParseError(s, types.Int, err)
}
case types.JsonFamily:
var j json.JSON
if j, err = json.ParseJSON(s); err == nil {
vh.JSON(j)
} else {
err = pgerror.Wrapf(err, pgcode.Syntax, "could not parse JSON")
}
case types.StringFamily:
s = truncateString(s, t)
vh.String(s)
case types.TimestampTZFamily:
// TODO(cucaroach): can we refactor the next 3 case arms to be simpler
// and avoid code duplication?
now := relativeParseTime(ctx)
var ts time.Time
if ts, _, err = pgdate.ParseTimestamp(now, dateStyle(ctx), s); err == nil {
// Always normalize time to the current location.
if ts, err = checkTimeBounds(ts, TimeFamilyPrecisionToRoundDuration(t.Precision())); err == nil {
vh.TimestampTZ(ts)
}
}
case types.TimestampFamily:
now := relativeParseTime(ctx)
var ts time.Time
if ts, _, err = pgdate.ParseTimestampWithoutTimezone(now, dateStyle(ctx), s); err == nil {
// Always normalize time to the current location.
if ts, err = checkTimeBounds(ts, TimeFamilyPrecisionToRoundDuration(t.Precision())); err == nil {
vh.TimestampTZ(ts)
}
}
case types.IntervalFamily:
var itm types.IntervalTypeMetadata
itm, err = t.IntervalTypeMetadata()
if err == nil {
var d duration.Duration
d, err = ParseIntervalWithTypeMetadata(intervalStyle(ctx), s, itm)
if err == nil {
vh.Duration(d)
}
}
case types.UuidFamily:
var uv uuid.UUID
uv, err = uuid.FromString(s)
if err == nil {
vh.Bytes(uv.GetBytes())
} else {
err = MakeParseError(s, types.Uuid, err)
}
default:
var d Datum
if d, _, err = ParseAndRequireString(t, s, ctx); err == nil {
vh.Datum(d)
}
}
return err
}

0 comments on commit 07debe9

Please sign in to comment.