From 7cdc27a5f7b1e99f4346aa05be3ae720cbe39931 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 30 Jan 2023 16:16:10 -0500 Subject: [PATCH] sem/tree: add support for producing vectorized data from strings 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 --- pkg/col/coldataext/BUILD.bazel | 6 + pkg/col/coldataext/vec_handler.go | 159 ++++++++++++++++++++ pkg/sql/sem/tree/BUILD.bazel | 5 + pkg/sql/sem/tree/datum.go | 27 +++- pkg/sql/sem/tree/parse_string.go | 162 +++++++++++++++++++- pkg/sql/sem/tree/parse_string_test.go | 206 ++++++++++++++++++++++++++ 6 files changed, 552 insertions(+), 13 deletions(-) create mode 100644 pkg/col/coldataext/vec_handler.go create mode 100644 pkg/sql/sem/tree/parse_string_test.go diff --git a/pkg/col/coldataext/BUILD.bazel b/pkg/col/coldataext/BUILD.bazel index 5f258b8e9882..1a98d29510fe 100644 --- a/pkg/col/coldataext/BUILD.bazel +++ b/pkg/col/coldataext/BUILD.bazel @@ -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"], @@ -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", ], ) diff --git a/pkg/col/coldataext/vec_handler.go b/pkg/col/coldataext/vec_handler.go new file mode 100644 index 000000000000..27990ddfa1b0 --- /dev/null +++ b/pkg/col/coldataext/vec_handler.go @@ -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++ +} diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 5ba3bcf963f6..ddd28ec8325f 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -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", @@ -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", @@ -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", diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index 9d917bf34410..e22206956920 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -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) } @@ -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) { @@ -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. @@ -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) { diff --git a/pkg/sql/sem/tree/parse_string.go b/pkg/sql/sem/tree/parse_string.go index c7edf20370b2..b11076d54269 100644 --- a/pkg/sql/sem/tree/parse_string.go +++ b/pkg/sql/sem/tree/parse_string.go @@ -13,15 +13,25 @@ package tree import ( "strconv" "strings" + "time" + "github.com/cockroachdb/apd/v3" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "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). @@ -74,12 +84,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())) @@ -115,6 +120,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) { @@ -143,3 +158,134 @@ 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() +} + +// ParseAndRequireStringHandler parses a string and passes values +// supported by the vector engine directly to a ValueHandler. Other types are +// handled by ParseAndRequireString. +func ParseAndRequireStringHandler( + 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) + } + case types.EnumFamily: + var d DEnum + d, err = MakeDEnumFromLogicalRepresentation(t, s) + if err == nil { + vh.Bytes(d.PhysicalRep) + } + default: + if typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) != typeconv.DatumVecCanonicalTypeFamily { + return errors.AssertionFailedf("unexpected type %v in datum case arm, does a new type need to be handled?", t) + } + var d Datum + if d, _, err = ParseAndRequireString(t, s, ctx); err == nil { + vh.Datum(d) + } + } + return err +} diff --git a/pkg/sql/sem/tree/parse_string_test.go b/pkg/sql/sem/tree/parse_string_test.go new file mode 100644 index 000000000000..39fd8b97d478 --- /dev/null +++ b/pkg/sql/sem/tree/parse_string_test.go @@ -0,0 +1,206 @@ +// 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 tree_test + +import ( + "testing" + "time" + + "github.com/cockroachdb/apd/v3" + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/colconv" + "github.com/cockroachdb/cockroach/pkg/sql/randgen" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "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/json" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" + "github.com/stretchr/testify/require" +) + +func vecRow(v coldata.Vec, i int) any { + switch v.CanonicalTypeFamily() { + case types.BoolFamily: + return v.Bool()[i] + case types.BytesFamily: + return v.Bytes().Get(i) + case types.IntFamily: + return v.Int64()[i] + case types.DecimalFamily: + return v.Decimal().Get(i) + case types.FloatFamily: + return v.Float64()[i] + case types.JsonFamily: + return v.JSON().Get(i) + case types.TimestampTZFamily: + return v.Timestamp()[i] + case types.IntervalFamily: + return v.Interval().Get(i) + default: + return v.Datum().Get(i) + } +} + +// TestParseStringTypeGamut is a sanity test for ParseAndRequireStringHandler +// that tests that all the basic scalar types parse to the same value as +// ParseAndRequireString. +func TestParseStringTypeGamut(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + factory := coldataext.NewExtendedColumnFactory(evalCtx) + b := coldata.NewMemBatchWithCapacity(types.Scalar, 2, factory) + vecHandlers := make([]tree.ValueHandler, len(types.Scalar)) + rng, _ := randutil.NewTestRand() + for i, typ := range types.Scalar { + d := randgen.RandDatum(rng, typ, false) + s := d.String() + // ParseAndRequireString doesn't like things wrapped in single quotes + // (its raw data not SQL). + if s[0] == '\'' { + s = s[1 : len(s)-1] + } + d, _, err1 := tree.ParseAndRequireString(typ, s, evalCtx) + + vecHandlers[i] = coldataext.MakeVecHandler(b.ColVec(i)) + err2 := tree.ParseAndRequireStringHandler(typ, s, evalCtx, vecHandlers[i], &evalCtx.ParseHelper) + require.Equal(t, err1, err2) + if err1 == nil { + converter := colconv.GetDatumToPhysicalFn(typ) + coldata.SetValueAt(b.ColVec(i), converter(d), 1 /* rowIdx */) + // ParseAndRequireStringHandler set the first row and second was converted datum, + // test that they are equal. + require.Equal(t, vecRow(b.ColVec(i), 0), vecRow(b.ColVec(i), 1)) + } + } +} + +// TestParseStringHandlerErrors tests that bogus strings return the same errors from +// ParseAndRequireString and ParseAndRequireStringHandler for vector types. +func TestParseStringHandlerErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + + for _, tc := range []struct { + val string + t *types.T + }{ + {"", types.Bool}, + {"o", types.Bool}, + {`\xa`, types.Bytes}, + {"", types.Date}, + {"423136fs", types.Date}, + {"", types.Decimal}, + {"not a decimal", types.Decimal}, + {"", types.Float}, + {"abc", types.Float}, + {"", types.Int}, + {"abc", types.Int}, + {"", types.Json}, + {"}{", types.Json}, + {"", types.TimestampTZ}, + {"not a timestamp", types.TimestampTZ}, + {"", types.Timestamp}, + {"not a timestamp", types.Timestamp}, + {"", types.Interval}, + {"Pasdf", types.Interval}, + {"", types.Uuid}, + {"notuuid", types.Uuid}, + } { + _, _, err1 := tree.ParseAndRequireString(tc.t, tc.val, evalCtx) + require.Error(t, err1, "parsing `%s` as `%v` didn't error as expected", tc.val, tc.t) + vh := &anyHandler{} + err2 := tree.ParseAndRequireStringHandler(tc.t, tc.val, evalCtx, vh, &evalCtx.ParseHelper) + require.Equal(t, err1.Error(), err2.Error()) + } +} + +type anyHandler struct { + val any + dec apd.Decimal +} + +var _ tree.ValueHandler = (*anyHandler)(nil) + +func (a *anyHandler) Len() int { return 0 } +func (a *anyHandler) Null() { a.val = nil } +func (a *anyHandler) Date(d pgdate.Date) { a.val = d } +func (a *anyHandler) Datum(d tree.Datum) { a.val = d } +func (a *anyHandler) Bool(b bool) { a.val = b } +func (a *anyHandler) Bytes(b []byte) { a.val = b } +func (a *anyHandler) Decimal() *apd.Decimal { return &a.dec } +func (a *anyHandler) Float(f float64) { a.val = f } +func (a *anyHandler) Int(i int64) { a.val = i } +func (a *anyHandler) Duration(d duration.Duration) { a.val = d } +func (a *anyHandler) JSON(j json.JSON) { a.val = j } +func (a *anyHandler) String(s string) { a.val = s } +func (a *anyHandler) TimestampTZ(t time.Time) { a.val = t } +func (a *anyHandler) Reset() {} + +type benchCase struct { + typ *types.T + str string +} + +var benchCases []benchCase = []benchCase{ + {types.Date, "1996-03-13"}, + {types.Bool, "true"}, + {types.Decimal, "21168.23"}, + {types.Float, "0.04"}, + {types.Int, "155190"}, + {types.Interval, "1h"}, + {types.Json, `{"a": "b"}`}, + {types.String, "asdf"}, + {types.TimestampTZ, "2000-05-05 10:00:00+03"}, +} + +func BenchmarkParseString(b *testing.B) { + evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings()) + factory := coldataext.NewExtendedColumnFactory(evalCtx) + numRows := 1000 + var typs = make([]*types.T, len(benchCases)) + for i, tc := range benchCases { + typs[i] = tc.typ + } + for _, tc := range benchCases { + b.Run(tc.typ.Name(), func(b *testing.B) { + for i := 0; i < b.N; i++ { + tree.ParseAndRequireString(tc.typ, tc.str, evalCtx) + } + }) + } + for col, tc := range benchCases { + b.Run("vec/"+tc.typ.Name(), func(b *testing.B) { + var vhs = make([]tree.ValueHandler, len(benchCases)) + ba := coldata.NewMemBatchWithCapacity(typs, numRows, factory) + for i := range benchCases { + vhs[i] = coldataext.MakeVecHandler(ba.ColVec(i)) + } + b.ResetTimer() + rowCount := 0 + for i := 0; i < b.N; i++ { + tree.ParseAndRequireStringHandler(tc.typ, tc.str, evalCtx, vhs[col], &evalCtx.ParseHelper) + rowCount++ + if rowCount == numRows { + vhs[col].Reset() + rowCount = 0 + } + } + }) + } +}