diff --git a/Makefile b/Makefile index 0b83fbb8de04..c400a1467840 100644 --- a/Makefile +++ b/Makefile @@ -1533,13 +1533,13 @@ bin/.docgen_functions: bin/docgen docs/generated/redact_safe.md: @(echo "The following types are considered always safe for reporting:"; echo; \ - echo "File | Type"; echo "--|--") >$@.tmp - @git grep '^func \(.*\) SafeValue\(\)' | \ + echo "File | Type"; echo "--|--") >$@.tmp || { rm -f $@.tmp; exit 1; } + @git grep -n '^func \(.*\) SafeValue\(\)' | \ grep -v '^pkg/util/redact' | \ - sed -E -e 's/^([^:]*):func \(([^ ]* )?(.*)\) SafeValue.*$$/\1 | \`\3\`/g' >>$@.tmp || rm -f $@.tmp - @git grep 'redact\.RegisterSafeType' | \ + sed -E -e 's/^([^:]*):[0-9]+:func \(([^ ]* )?(.*)\) SafeValue.*$$/\1 | \`\3\`/g' >>$@.tmp || { rm -f $@.tmp; exit 1; } + @git grep -n 'redact\.RegisterSafeType' | \ grep -v '^pkg/util/redact' | \ - sed -E -e 's/^([^:]*):.*redact\.RegisterSafeType\((.*)\).*/\1 | \`\2\`/g' >>$@.tmp || rm -f $@.tmp + sed -E -e 's/^([^:]*):[0-9]+:.*redact\.RegisterSafeType\((.*)\).*/\1 | \`\2\`/g' >>$@.tmp || { rm -f $@.tmp; exit 1; } @mv -f $@.tmp $@ settings-doc-gen := $(if $(filter buildshort,$(MAKECMDGOALS)),$(COCKROACHSHORT),$(COCKROACH)) diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 558328a8ef42..f10b23e31ec6 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -776,6 +776,9 @@ has no relationship with the commit order of concurrent transactions.

st_astext(geometry: geometry, maximum_decimal_digits: int) → string

Returns the WKT representation of a given Geometry. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the .. Use -1 to print as many digits as possible.

+st_azimuth(geometry_a: geometry, geometry_b: geometry) → float

Returns the azimuth in radians of the segment defined by the given point geometries, or NULL if the two points are coincident.

+

The azimuth is angle is referenced from north, and is positive clockwise: North = 0; East = π/2; South = π; West = 3π/2.

+
st_buffer(geometry: geometry, distance: float) → geometry

Returns a Geometry that represents all points whose distance is less than or equal to the given distance from the given Geometry.

This function utilizes the GEOS module.

diff --git a/pkg/col/coldataext/datum_vec.go b/pkg/col/coldataext/datum_vec.go index a46646a5e5a7..c6d5eac49265 100644 --- a/pkg/col/coldataext/datum_vec.go +++ b/pkg/col/coldataext/datum_vec.go @@ -70,6 +70,12 @@ func (d *Datum) CompareDatum(dVec, other interface{}) int { return d.Datum.Compare(dVec.(*datumVec).evalCtx, maybeUnwrapDatum(other)) } +// Cast returns the result of casting d to the type toType. dVec is the +// datumVec that stores d and is used to supply the eval context. +func (d *Datum) Cast(dVec interface{}, toType *types.T) (tree.Datum, error) { + return tree.PerformCast(dVec.(*datumVec).evalCtx, d.Datum, toType) +} + // Hash returns the hash of the datum as a byte slice. func (d *Datum) Hash(da *sqlbase.DatumAlloc) []byte { ed := sqlbase.EncDatum{Datum: maybeUnwrapDatum(d)} diff --git a/pkg/geo/geomfn/azimuth.go b/pkg/geo/geomfn/azimuth.go new file mode 100644 index 000000000000..ec46ab8377fe --- /dev/null +++ b/pkg/geo/geomfn/azimuth.go @@ -0,0 +1,58 @@ +// 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 geomfn + +import ( + "math" + + "github.com/cockroachdb/cockroach/pkg/geo" + "github.com/cockroachdb/errors" + "github.com/twpayne/go-geom" +) + +// Azimuth returns the azimuth in radians of the segment defined by the given point geometries. +// The azimuth is angle is referenced from north, and is positive clockwise. +// North = 0; East = π/2; South = π; West = 3π/2. +// Returns nil if the two points are the same. +// Returns an error if any of the two Geometry items are not points. +func Azimuth(a *geo.Geometry, b *geo.Geometry) (*float64, error) { + aGeomT, err := a.AsGeomT() + if err != nil { + return nil, err + } + + aPoint, ok := aGeomT.(*geom.Point) + if !ok { + return nil, errors.Newf("Argument must be POINT geometries") + } + + bGeomT, err := b.AsGeomT() + if err != nil { + return nil, err + } + + bPoint, ok := bGeomT.(*geom.Point) + if !ok { + return nil, errors.Newf("Argument must be POINT geometries") + } + + if aPoint.X() == bPoint.X() && aPoint.Y() == bPoint.Y() { + return nil, nil + } + + atan := math.Atan2(bPoint.Y()-aPoint.Y(), bPoint.X()-aPoint.X()) // Started at East(90) counterclockwise. + const degree360 = 2 * math.Pi // Added 360 degrees for always returns a positive value. + const degree90 = math.Pi / 2 // Added 90 degrees to get it started at North(0). + + azimuth := math.Mod(degree360+degree90-atan, 2*math.Pi) + + return &azimuth, nil +} diff --git a/pkg/geo/geomfn/azimuth_test.go b/pkg/geo/geomfn/azimuth_test.go new file mode 100644 index 000000000000..30fc51f4fb4b --- /dev/null +++ b/pkg/geo/geomfn/azimuth_test.go @@ -0,0 +1,76 @@ +// 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 geomfn + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/geo" + "github.com/stretchr/testify/require" +) + +func TestAzimuth(t *testing.T) { + zero := 0.0 + aQuarterPi := 0.7853981633974483 + towQuartersPi := 1.5707963267948966 + threeQuartersPi := 2.356194490192344 + + testCases := []struct { + a string + b string + expected *float64 + }{ + { + "POINT(0 0)", + "POINT(0 0)", + nil, + }, + { + "POINT(0 0)", + "POINT(1 1)", + &aQuarterPi, + }, + { + "POINT(0 0)", + "POINT(1 0)", + &towQuartersPi, + }, + { + "POINT(0 0)", + "POINT(1 -1)", + &threeQuartersPi, + }, + { + "POINT(0 0)", + "POINT(0 1)", + &zero, + }, + } + + for i, tc := range testCases { + t.Run(fmt.Sprintf("tc:%d", i), func(t *testing.T) { + a, err := geo.ParseGeometry(tc.a) + require.NoError(t, err) + b, err := geo.ParseGeometry(tc.b) + require.NoError(t, err) + + r, err := Azimuth(a, b) + require.NoError(t, err) + + if tc.expected == nil { + require.Nil(t, r) + } else { + require.Equal(t, *tc.expected, *r) + } + }) + } +} diff --git a/pkg/kv/kvserver/main_test.go b/pkg/kv/kvserver/main_test.go index 6dd7c3828250..19df816aa3ca 100644 --- a/pkg/kv/kvserver/main_test.go +++ b/pkg/kv/kvserver/main_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) @@ -40,6 +41,15 @@ func TestMain(m *testing.M) { serverutils.InitTestServerFactory(server.TestServerFactory) serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + // The below-Raft proto tracking is fairly expensive in terms of allocations + // which significantly impacts the tests under -race. We're already doing the + // below-Raft proto tracking in non-race builds, so there is little benefit + // to also doing it in race builds. + if util.RaceEnabled { + os.Exit(m.Run()) + return + } + // Create a set of all protos we believe to be marshaled downstream of raft. // After the tests are run, we'll subtract the encountered protos from this // set. diff --git a/pkg/kv/kvserver/replica_sideload_test.go b/pkg/kv/kvserver/replica_sideload_test.go index ee52ed4ed142..ec09e840aced 100644 --- a/pkg/kv/kvserver/replica_sideload_test.go +++ b/pkg/kv/kvserver/replica_sideload_test.go @@ -290,8 +290,7 @@ func testSideloadingSideloadedStorage( if err == nil { t.Fatalf("sideloaded directory should not have been removable due to extra file %s", nonRemovableFile) } - expectedTruncateError := "while purging %q: remove %s: directory not empty" - if err.Error() != fmt.Sprintf(expectedTruncateError, ss.(*diskSideloadStorage).dir, ss.(*diskSideloadStorage).dir) { + if !strings.HasSuffix(strings.ToLower(err.Error()), "directory not empty") { t.Fatalf("error truncating sideloaded storage: %+v", err) } // Now remove extra file and let truncation proceed to remove directory. diff --git a/pkg/sql/colexec/bool_vec_to_sel.go b/pkg/sql/colexec/bool_vec_to_sel.go index 040a3ddc125b..50cfc1513ee4 100644 --- a/pkg/sql/colexec/bool_vec_to_sel.go +++ b/pkg/sql/colexec/bool_vec_to_sel.go @@ -31,7 +31,7 @@ func boolOrUnknownToSelOp( // If the column is of an Unknown type, then all values in that column // must be NULLs, so the selection vector will always be empty, and we // can simply plan a zero operator. - return NewZeroOp(input), nil + return newZeroOp(input), nil default: return nil, errors.Errorf("unexpectedly %s is neither bool nor unknown", typs[vecIdx]) } diff --git a/pkg/sql/colexec/cast_test.go b/pkg/sql/colexec/cast_test.go index 13172f645e0c..1cd164deee68 100644 --- a/pkg/sql/colexec/cast_test.go +++ b/pkg/sql/colexec/cast_test.go @@ -40,6 +40,7 @@ func TestRandomizedCast(t *testing.T) { Settings: st, }, } + rng, _ := randutil.NewPseudoRand() datumAsBool := func(d tree.Datum) interface{} { return bool(tree.MustBeDBool(d)) @@ -53,36 +54,65 @@ func TestRandomizedCast(t *testing.T) { datumAsDecimal := func(d tree.Datum) interface{} { return tree.MustBeDDecimal(d).Decimal } + datumAsColdataextDatum := func(datumVec coldata.DatumVec, d tree.Datum) interface{} { + datumVec.Set(0, d) + return datumVec.Get(0) + } + makeDatumVecAdapter := func(datumVec coldata.DatumVec) func(tree.Datum) interface{} { + return func(d tree.Datum) interface{} { + return datumAsColdataextDatum(datumVec, d) + } + } + + collatedStringType := types.MakeCollatedString(types.String, *sqlbase.RandCollationLocale(rng)) + collatedStringVec := testColumnFactory.MakeColumn(collatedStringType, 1 /* n */).(coldata.DatumVec) + getCollatedStringsThatCanBeCastAsBools := func() []tree.Datum { + var res []tree.Datum + for _, validString := range []string{"true", "false", "yes", "no"} { + d, err := tree.NewDCollatedString(validString, collatedStringType.Locale(), &tree.CollationEnvironment{}) + if err != nil { + t.Fatal(err) + } + res = append(res, d) + } + return res + } tc := []struct { fromTyp *types.T fromPhysType func(tree.Datum) interface{} toTyp *types.T toPhysType func(tree.Datum) interface{} - // Some types casting can fail, so retry if we - // generate a datum that is unable to be casted. + // getValidSet (when non-nil) is a function that returns a set of valid + // datums of fromTyp type that can be cast to toTyp type. The test + // harness will be randomly choosing a datum from this set. This + // function should be specified when sqlbase.RandDatum will take ages + // (if ever) to generate the datum that is valid for a cast. + getValidSet func() []tree.Datum + // Some types casting can fail, so retry if we generate a datum that is + // unable to be cast. retryGeneration bool }{ //bool -> t tests - {types.Bool, datumAsBool, types.Bool, datumAsBool, false}, - {types.Bool, datumAsBool, types.Int, datumAsInt, false}, - {types.Bool, datumAsBool, types.Float, datumAsFloat, false}, + {fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Bool, toPhysType: datumAsBool}, + {fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Int, toPhysType: datumAsInt}, + {fromTyp: types.Bool, fromPhysType: datumAsBool, toTyp: types.Float, toPhysType: datumAsFloat}, // decimal -> t tests - {types.Decimal, datumAsDecimal, types.Bool, datumAsBool, false}, + {fromTyp: types.Decimal, fromPhysType: datumAsDecimal, toTyp: types.Bool, toPhysType: datumAsBool}, // int -> t tests - {types.Int, datumAsInt, types.Bool, datumAsBool, false}, - {types.Int, datumAsInt, types.Float, datumAsFloat, false}, - {types.Int, datumAsInt, types.Decimal, datumAsDecimal, false}, + {fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Bool, toPhysType: datumAsBool}, + {fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Float, toPhysType: datumAsFloat}, + {fromTyp: types.Int, fromPhysType: datumAsInt, toTyp: types.Decimal, toPhysType: datumAsDecimal}, // float -> t tests - {types.Float, datumAsFloat, types.Bool, datumAsBool, false}, + {fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Bool, toPhysType: datumAsBool}, // We can sometimes generate a float outside of the range of the integers, // so we want to retry with generation if that occurs. - {types.Float, datumAsFloat, types.Int, datumAsInt, true}, - {types.Float, datumAsFloat, types.Decimal, datumAsDecimal, false}, + {fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Int, toPhysType: datumAsInt, retryGeneration: true}, + {fromTyp: types.Float, fromPhysType: datumAsFloat, toTyp: types.Decimal, toPhysType: datumAsDecimal}, + // datum-backed type -> t tests + {fromTyp: collatedStringType, fromPhysType: makeDatumVecAdapter(collatedStringVec), toTyp: types.Bool, toPhysType: datumAsBool, getValidSet: getCollatedStringsThatCanBeCastAsBools}, } - rng, _ := randutil.NewPseudoRand() - for _, c := range tc { t.Run(fmt.Sprintf("%sTo%s", c.fromTyp.String(), c.toTyp.String()), func(t *testing.T) { n := 100 @@ -90,29 +120,34 @@ func TestRandomizedCast(t *testing.T) { input := tuples{} output := tuples{} for i := 0; i < n; i++ { - // We don't allow any NULL datums to be generated, so disable - // this ability in the RandDatum function. - fromDatum := sqlbase.RandDatum(rng, c.fromTyp, false) var ( - toDatum tree.Datum - err error + fromDatum, toDatum tree.Datum + err error ) - toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp) - if c.retryGeneration { - for err != nil { - // If we are allowed to retry, make a new datum and cast it on error. - fromDatum = sqlbase.RandDatum(rng, c.fromTyp, false) - toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp) - } + if c.getValidSet != nil { + validFromDatums := c.getValidSet() + fromDatum = validFromDatums[rng.Intn(len(validFromDatums))] + toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp) } else { - if err != nil { - t.Fatal(err) + // We don't allow any NULL datums to be generated, so disable + // this ability in the RandDatum function. + fromDatum = sqlbase.RandDatum(rng, c.fromTyp, false) + toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp) + if c.retryGeneration { + for err != nil { + // If we are allowed to retry, make a new datum and cast it on error. + fromDatum = sqlbase.RandDatum(rng, c.fromTyp, false) + toDatum, err = tree.PerformCast(&evalCtx, fromDatum, c.toTyp) + } } } + if err != nil { + t.Fatal(err) + } input = append(input, tuple{c.fromPhysType(fromDatum)}) output = append(output, tuple{c.fromPhysType(fromDatum), c.toPhysType(toDatum)}) } - runTests(t, []tuples{input}, output, orderedVerifier, + runTestsWithTyps(t, []tuples{input}, [][]*types.T{{c.fromTyp}}, output, orderedVerifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { return createTestCastOperator(ctx, flowCtx, input[0], c.fromTyp, c.toTyp) }) diff --git a/pkg/sql/colexec/cast_tmpl.go b/pkg/sql/colexec/cast_tmpl.go index ff08c5a8ae50..37f97533bf7c 100644 --- a/pkg/sql/colexec/cast_tmpl.go +++ b/pkg/sql/colexec/cast_tmpl.go @@ -56,7 +56,7 @@ const _RIGHT_CANONICAL_TYPE_FAMILY = types.UnknownFamily // _RIGHT_TYPE_WIDTH is the template variable. const _RIGHT_TYPE_WIDTH = 0 -func _CAST(to, from interface{}) { +func _CAST(to, from, fromCol interface{}) { colexecerror.InternalError("") } @@ -104,7 +104,7 @@ func cast(inputVec, outputVec coldata.Vec, n int, sel []int) { } else { v := _L_UNSAFEGET(inputCol, i) var r _R_GO_TYPE - _CAST(r, v) + _CAST(r, v, inputCol) _R_SET(outputCol, i, r) } } @@ -116,7 +116,7 @@ func cast(inputVec, outputVec coldata.Vec, n int, sel []int) { } else { v := _L_UNSAFEGET(inputCol, i) var r _R_GO_TYPE - _CAST(r, v) + _CAST(r, v, inputCol) _R_SET(outputCol, i, r) } } @@ -127,7 +127,7 @@ func cast(inputVec, outputVec coldata.Vec, n int, sel []int) { for _, i := range sel { v := _L_UNSAFEGET(inputCol, i) var r _R_GO_TYPE - _CAST(r, v) + _CAST(r, v, inputCol) _R_SET(outputCol, i, r) } } else { @@ -135,7 +135,7 @@ func cast(inputVec, outputVec coldata.Vec, n int, sel []int) { for execgen.RANGE(i, inputCol, 0, n) { v := _L_UNSAFEGET(inputCol, i) var r _R_GO_TYPE - _CAST(r, v) + _CAST(r, v, inputCol) _R_SET(outputCol, i, r) } } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/cast_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/cast_gen.go index 4dc444480ed2..fa6a80ca83cc 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/cast_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/cast_gen.go @@ -30,8 +30,8 @@ func genCastOperators(inputFileContents string, wr io.Writer) error { ) s := r.Replace(inputFileContents) - castRe := makeFunctionRegex("_CAST", 2) - s = castRe.ReplaceAllString(s, makeTemplateFunctionCall("Right.Cast", 2)) + castRe := makeFunctionRegex("_CAST", 3) + s = castRe.ReplaceAllString(s, makeTemplateFunctionCall("Right.Cast", 3)) s = strings.ReplaceAll(s, "_L_SLICE", "execgen.SLICE") s = strings.ReplaceAll(s, "_L_UNSAFEGET", "execgen.UNSAFEGET") diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go index 1bd4b8c47842..b05a5a4a4dcc 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go @@ -344,7 +344,7 @@ type twoArgsResolvedOverloadRightWidthInfo struct { type assignFunc func(op *lastArgWidthOverload, targetElem, leftElem, rightElem, targetCol, leftCol, rightCol string) string type compareFunc func(targetElem, leftElem, rightElem, leftCol, rightCol string) string -type castFunc func(to, from string) string +type castFunc func(to, from, fromCol string) string // Assign produces a Go source string that assigns the "targetElem" variable to // the result of applying the overload to the two inputs, "leftElem" and @@ -392,9 +392,9 @@ func (o *lastArgWidthOverload) Compare( leftElem, rightElem, targetElem, leftElem, rightElem, targetElem, targetElem) } -func (o *lastArgWidthOverload) Cast(to, from string) string { +func (o *lastArgWidthOverload) Cast(to, from, fromCol string) string { if o.CastFunc != nil { - if ret := o.CastFunc(to, from); ret != "" { + if ret := o.CastFunc(to, from, fromCol); ret != "" { return ret } } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go index f0cf1ce3b85e..ac2cbb4378ce 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go @@ -13,6 +13,7 @@ package main import ( "fmt" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -56,15 +57,15 @@ func populateCastOverloads() { }, castTypeCustomizers) } -func intToDecimal(to, from string) string { +func intToDecimal(to, from, _ string) string { convStr := ` %[1]s = *apd.New(int64(%[2]s), 0) ` return fmt.Sprintf(convStr, to, from) } -func intToFloat() func(string, string) string { - return func(to, from string) string { +func intToFloat() func(string, string, string) string { + return func(to, from, _ string) string { convStr := ` %[1]s = float64(%[2]s) ` @@ -72,29 +73,29 @@ func intToFloat() func(string, string) string { } } -func intToInt16(to, from string) string { +func intToInt16(to, from, _ string) string { convStr := ` %[1]s = int16(%[2]s) ` return fmt.Sprintf(convStr, to, from) } -func intToInt32(to, from string) string { +func intToInt32(to, from, _ string) string { convStr := ` %[1]s = int32(%[2]s) ` return fmt.Sprintf(convStr, to, from) } -func intToInt64(to, from string) string { +func intToInt64(to, from, _ string) string { convStr := ` %[1]s = int64(%[2]s) ` return fmt.Sprintf(convStr, to, from) } -func floatToInt(intWidth, floatWidth int32) func(string, string) string { - return func(to, from string) string { +func floatToInt(intWidth, floatWidth int32) func(string, string, string) string { + return func(to, from, _ string) string { convStr := ` if math.IsNaN(float64(%[2]s)) || %[2]s <= float%[4]d(math.MinInt%[3]d) || %[2]s >= float%[4]d(math.MaxInt%[3]d) { colexecerror.ExpectedError(tree.ErrIntOutOfRange) @@ -108,14 +109,14 @@ func floatToInt(intWidth, floatWidth int32) func(string, string) string { } } -func numToBool(to, from string) string { +func numToBool(to, from, _ string) string { convStr := ` %[1]s = %[2]s != 0 ` return fmt.Sprintf(convStr, to, from) } -func floatToDecimal(to, from string) string { +func floatToDecimal(to, from, _ string) string { convStr := ` { var tmpDec apd.Decimal @@ -129,6 +130,19 @@ func floatToDecimal(to, from string) string { return fmt.Sprintf(convStr, to, from) } +func datumToBool(to, from, fromCol string) string { + convStr := ` + { + _castedDatum, err := %[2]s.(*coldataext.Datum).Cast(%[3]s, types.Bool) + if err != nil { + colexecerror.ExpectedError(err) + } + %[1]s = _castedDatum == tree.DBoolTrue + } + ` + return fmt.Sprintf(convStr, to, from, fromCol) +} + // castTypeCustomizer is a type customizer that changes how the templater // produces cast operator output for a particular type. type castTypeCustomizer interface { @@ -169,7 +183,8 @@ func registerCastTypeCustomizers() { for _, intWidth := range supportedWidthsByCanonicalTypeFamily[types.IntFamily] { registerCastTypeCustomizer(typePair{types.IntFamily, intWidth, types.IntFamily, intWidth}, intCustomizer{width: intWidth}) } - // TODO(yuzefovich): add casts for Timestamps and Intervals. + // TODO(yuzefovich): add casts for Timestamps, Intervals, and datum-backed + // types. // Casts from boolean. registerCastTypeCustomizer(typePair{types.BoolFamily, anyWidth, types.FloatFamily, anyWidth}, boolCastCustomizer{}) @@ -200,6 +215,9 @@ func registerCastTypeCustomizers() { registerCastTypeCustomizer(typePair{types.FloatFamily, anyWidth, toFamily, toWidth}, floatCastCustomizer{toFamily: toFamily, toWidth: toWidth}) } } + + // Casts from datum-backed types. + registerCastTypeCustomizer(typePair{typeconv.DatumVecCanonicalTypeFamily, anyWidth, types.BoolFamily, anyWidth}, datumCastCustomizer{toFamily: types.BoolFamily}) } // boolCastCustomizer specifies casts from booleans. @@ -220,8 +238,14 @@ type intCastCustomizer struct { toWidth int32 } +// datumCastCustomizer specifies casts from types that are backed by tree.Datum +// to other types. +type datumCastCustomizer struct { + toFamily types.Family +} + func (boolCastCustomizer) getCastFunc() castFunc { - return func(to, from string) string { + return func(to, from, _ string) string { convStr := ` %[1]s = 0 if %[2]s { @@ -233,7 +257,7 @@ func (boolCastCustomizer) getCastFunc() castFunc { } func (decimalCastCustomizer) getCastFunc() castFunc { - return func(to, from string) string { + return func(to, from, _ string) string { return fmt.Sprintf("%[1]s = %[2]s.Sign() != 0", to, from) } } @@ -274,3 +298,13 @@ func (c intCastCustomizer) getCastFunc() castFunc { // This code is unreachable, but the compiler cannot infer that. return nil } + +func (c datumCastCustomizer) getCastFunc() castFunc { + switch c.toFamily { + case types.BoolFamily: + return datumToBool + } + colexecerror.InternalError(fmt.Sprintf("unexpectedly didn't find a cast from datum-backed type to %s", c.toFamily)) + // This code is unreachable, but the compiler cannot infer that. + return nil +} diff --git a/pkg/sql/colexec/execplan.go b/pkg/sql/colexec/execplan.go index 063baff2b567..fe6a706c0bd2 100644 --- a/pkg/sql/colexec/execplan.go +++ b/pkg/sql/colexec/execplan.go @@ -192,6 +192,12 @@ func isSupported(mode sessiondata.VectorizeExecMode, spec *execinfrapb.Processor case core.Noop != nil: return nil + case core.Values != nil: + if core.Values.NumRows != 0 { + return errors.Newf("values core only with zero rows supported") + } + return nil + case core.TableReader != nil: if core.TableReader.IsCheck { return errors.Newf("scrub table reader is unsupported in vectorized") @@ -631,6 +637,20 @@ func NewColOperator( result.Op, result.IsStreaming = NewNoop(inputs[0]), true result.ColumnTypes = make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(result.ColumnTypes, spec.Input[0].ColumnTypes) + + case core.Values != nil: + if err := checkNumIn(inputs, 0); err != nil { + return result, err + } + if core.Values.NumRows != 0 { + return result, errors.AssertionFailedf("values core only with zero rows supported, %d given", core.Values.NumRows) + } + result.Op, result.IsStreaming = NewZeroOpNoInput(), true + result.ColumnTypes = make([]*types.T, len(core.Values.Columns)) + for i, col := range core.Values.Columns { + result.ColumnTypes[i] = col.Type + } + case core.TableReader != nil: if err := checkNumIn(inputs, 0); err != nil { return result, err @@ -1368,7 +1388,7 @@ func (r *postProcessResult) planFilterExpr( if expr == tree.DNull { // The filter expression is tree.DNull meaning that it is always false, so // we put a zero operator. - r.Op = NewZeroOp(r.Op) + r.Op = newZeroOp(r.Op) return nil } var filterColumnTypes []*types.T diff --git a/pkg/sql/colexec/operator.go b/pkg/sql/colexec/operator.go index c94ad8712e3f..e3781acc9df0 100644 --- a/pkg/sql/colexec/operator.go +++ b/pkg/sql/colexec/operator.go @@ -185,8 +185,8 @@ type zeroOperator struct { var _ colexecbase.Operator = &zeroOperator{} -// NewZeroOp creates a new operator which just returns an empty batch. -func NewZeroOp(input colexecbase.Operator) colexecbase.Operator { +// newZeroOp creates a new operator which just returns an empty batch. +func newZeroOp(input colexecbase.Operator) colexecbase.Operator { return &zeroOperator{OneInputNode: NewOneInputNode(input)} } @@ -198,6 +198,25 @@ func (s *zeroOperator) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch } +type zeroOperatorNoInput struct { + colexecbase.ZeroInputNode + NonExplainable +} + +var _ colexecbase.Operator = &zeroOperatorNoInput{} + +// NewZeroOpNoInput creates a new operator which just returns an empty batch +// and doesn't an input. +func NewZeroOpNoInput() colexecbase.Operator { + return &zeroOperatorNoInput{} +} + +func (s *zeroOperatorNoInput) Init() {} + +func (s *zeroOperatorNoInput) Next(ctx context.Context) coldata.Batch { + return coldata.ZeroBatch +} + type singleTupleNoInputOperator struct { colexecbase.ZeroInputNode NonExplainable diff --git a/pkg/sql/colexec/utils_test.go b/pkg/sql/colexec/utils_test.go index 5a7c110b8c76..151a7cb80c2f 100644 --- a/pkg/sql/colexec/utils_test.go +++ b/pkg/sql/colexec/utils_test.go @@ -603,20 +603,25 @@ func setColVal(vec coldata.Vec, idx int, val interface{}) { vec.Decimal()[idx].Set(decimalVal) } } else if canonicalTypeFamily == typeconv.DatumVecCanonicalTypeFamily { - switch vec.Type().Family() { - case types.JsonFamily: - if jsonStr, ok := val.(string); ok { - jobj, err := json.ParseJSON(jsonStr) - if err != nil { - colexecerror.InternalError( - fmt.Sprintf("unable to parse json object: %v: %v", jobj, err)) + switch v := val.(type) { + case *coldataext.Datum: + vec.Datum().Set(idx, v) + default: + switch vec.Type().Family() { + case types.JsonFamily: + if jsonStr, ok := val.(string); ok { + jobj, err := json.ParseJSON(jsonStr) + if err != nil { + colexecerror.InternalError( + fmt.Sprintf("unable to parse json object: %v: %v", jobj, err)) + } + vec.Datum().Set(idx, &tree.DJSON{JSON: jobj}) + } else if jobj, ok := val.(json.JSON); ok { + vec.Datum().Set(idx, &tree.DJSON{JSON: jobj}) } - vec.Datum().Set(idx, &tree.DJSON{JSON: jobj}) - } else if jobj, ok := val.(json.JSON); ok { - vec.Datum().Set(idx, &tree.DJSON{JSON: jobj}) + default: + colexecerror.InternalError(fmt.Sprintf("unexpected datum-backed type: %s", vec.Type())) } - default: - colexecerror.InternalError(fmt.Sprintf("unexpected datum-backed type: %s", vec.Type())) } } else { reflect.ValueOf(vec.Col()).Index(idx).Set(reflect.ValueOf(val).Convert(reflect.TypeOf(vec.Col()).Elem())) @@ -997,16 +1002,18 @@ func getTupleFromBatch(batch coldata.Batch, tupleIdx int) tuple { newDec.Set(&colDec[tupleIdx]) val = reflect.ValueOf(newDec) } else if vec.CanonicalTypeFamily() == typeconv.DatumVecCanonicalTypeFamily { - switch vec.Type().Family() { - case types.JsonFamily: - d := vec.Datum().Get(tupleIdx).(*coldataext.Datum).Datum - if d == tree.DNull { - val = reflect.ValueOf(tree.DNull) - } else { - val = reflect.ValueOf(d.(*tree.DJSON).JSON) + d := vec.Datum().Get(tupleIdx).(*coldataext.Datum) + if d.Datum == tree.DNull { + val = reflect.ValueOf(tree.DNull) + } else { + switch vec.Type().Family() { + case types.CollatedStringFamily: + val = reflect.ValueOf(d) + case types.JsonFamily: + val = reflect.ValueOf(d.Datum.(*tree.DJSON).JSON) + default: + colexecerror.InternalError(fmt.Sprintf("unexpected datum-backed type: %s", vec.Type())) } - default: - colexecerror.InternalError(fmt.Sprintf("unexpected datum-backed type: %s", vec.Type())) } } else { val = reflect.ValueOf(vec.Col()).Index(tupleIdx) diff --git a/pkg/sql/colflow/vectorized_flow_space_test.go b/pkg/sql/colflow/vectorized_flow_space_test.go index 409d9c44b6cf..d759296ab62c 100644 --- a/pkg/sql/colflow/vectorized_flow_space_test.go +++ b/pkg/sql/colflow/vectorized_flow_space_test.go @@ -82,9 +82,9 @@ func TestVectorizeInternalMemorySpaceError(t *testing.T) { for _, tc := range testCases { for _, success := range []bool{true, false} { t.Run(fmt.Sprintf("%s-success-expected-%t", tc.desc, success), func(t *testing.T) { - inputs := []colexecbase.Operator{colexec.NewZeroOp(nil)} + inputs := []colexecbase.Operator{colexec.NewZeroOpNoInput()} if len(tc.spec.Input) > 1 { - inputs = append(inputs, colexec.NewZeroOp(nil)) + inputs = append(inputs, colexec.NewZeroOpNoInput()) } memMon := mon.MakeMonitor("MemoryMonitor", mon.MemoryResource, nil, nil, 0, math.MaxInt64, st) if success { diff --git a/pkg/sql/logictest/testdata/logic_test/geospatial b/pkg/sql/logictest/testdata/logic_test/geospatial index d8fa19f96403..21e8df7e4b1a 100644 --- a/pkg/sql/logictest/testdata/logic_test/geospatial +++ b/pkg/sql/logictest/testdata/logic_test/geospatial @@ -112,6 +112,21 @@ SELECT ST_AsText(ST_Project('POINT(0 0)'::geography, 100000, radians(45.0))) ---- POINT (0.635231029125537 0.639472334729198) +statement error Argument must be POINT geometries +SELECT ST_Azimuth('POLYGON((0 0, 0 0, 0 0, 0 0))'::geometry, 'POLYGON((0 0, 0 0, 0 0, 0 0))'::geometry) + +query RR +SELECT + degrees(ST_Azimuth(ST_Point(25, 45), ST_Point(75, 100))) AS degA_B, + degrees(ST_Azimuth(ST_Point(75, 100), ST_Point(25, 45))) AS degB_A +---- +42.2736890060937 222.273689006094 + +query R +SELECT ST_Azimuth(ST_Point(0, 0), ST_Point(0, 0)) +---- +NULL + subtest cast_test query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/enums b/pkg/sql/opt/exec/execbuilder/testdata/enums index 05d76e6ca15b..20e6e0366b4d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/enums +++ b/pkg/sql/opt/exec/execbuilder/testdata/enums @@ -31,20 +31,20 @@ query T EXPLAIN (OPT) SELECT * FROM t WHERE x > 'hello' ---- scan t - └── constraint: /1: [/'howdy' - ] + └── constraint: /1: [/'howdy' - /'hi'] # Test that we can perform constrained scans using secondary indexes too. query T EXPLAIN (OPT) SELECT * FROM t WHERE y = 'hello' ---- scan t@i - └── constraint: /2/1: [/'hello' - /'hello'] + └── constraint: /2/1: [/'hello'/'hello' - /'hello'/'hi'] query T EXPLAIN (OPT) SELECT * FROM t WHERE y > 'hello' AND y < 'hi' ---- scan t@i - └── constraint: /2/1: [/'howdy' - /'howdy'] + └── constraint: /2/1: [/'howdy'/'hello' - /'howdy'/'hi'] query T EXPLAIN (opt) SELECT * FROM t WHERE x IN ('hello', 'hi') @@ -53,3 +53,16 @@ scan t └── constraint: /1 ├── [/'hello' - /'hello'] └── [/'hi' - /'hi'] + +statement ok +CREATE TABLE checks (x greeting NOT NULL, y int, INDEX (x, y)) + +# Check that inferred check constraints from enum columns are used in plans. +query T +EXPLAIN (OPT) SELECT x, y FROM checks WHERE y = 2 +---- +scan checks@checks_x_y_idx + └── constraint: /1/2/3 + ├── [/'hello'/2 - /'hello'/2] + ├── [/'howdy'/2 - /'howdy'/2] + └── [/'hi'/2 - /'hi'/2] diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index b9f4f282662a..450d3d78c4da 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -538,6 +538,11 @@ type optTable struct { outboundFKs []optForeignKeyConstraint inboundFKs []optForeignKeyConstraint + // checkConstraints is the set of check constraints for this table. It + // can be different from desc's constraints because of synthesized + // constraints for user defined types. + checkConstraints []cat.CheckConstraint + // colMap is a mapping from unique ColumnID to column ordinal within the // table. This is a common lookup that needs to be fast. colMap map[sqlbase.ColumnID]int @@ -626,6 +631,55 @@ func newOptTable( ot.families[i].init(ot, &desc.Families[i+1]) } + // Synthesize any check constraints for user defined types. + var synthesizedChecks []cat.CheckConstraint + // TODO (rohany): We don't allow referencing columns in mutations in these + // expressions. However, it seems like we will need to have these checks + // operate on columns in mutations. Consider the following case: + // * a user adds a column with an enum type. + // * the column has a default expression of an enum that is not in the + // writeable state. + // * We will need a check constraint here to ensure that writes to the + // column are not successful, but we wouldn't be able to add that now. + for i := 0; i < ot.ColumnCount(); i++ { + col := ot.Column(i) + colType := col.DatumType() + if colType.UserDefined() { + switch colType.Family() { + case types.EnumFamily: + // TODO (rohany): When we can alter types, this logic will change. + // In particular, we will want to generate two check constraints if the + // enum contains values that are read only. The first constraint will + // be validated, and contain all of the members of the enum. The second + // will be unvalidated, and will contain only the writeable members of + // the enum. The unvalidated constraint ensures that only writeable + // members of the enum are written. The validated constraint ensures + // that all potentially written values of the enum are considered when + // planning read operations. + // We synthesize an (x IN (v1, v2, v3...)) check for enum types. + expr := &tree.ComparisonExpr{ + Operator: tree.In, + Left: &tree.ColumnItem{ColumnName: col.ColName()}, + Right: tree.NewDTuple(colType, tree.MakeAllDEnumsInType(colType)...), + } + synthesizedChecks = append(synthesizedChecks, cat.CheckConstraint{ + Constraint: tree.Serialize(expr), + Validated: true, + }) + } + } + } + // Move all existing and synthesized checks into the opt table. + activeChecks := desc.ActiveChecks() + ot.checkConstraints = make([]cat.CheckConstraint, 0, len(activeChecks)+len(synthesizedChecks)) + for i := range activeChecks { + ot.checkConstraints = append(ot.checkConstraints, cat.CheckConstraint{ + Constraint: activeChecks[i].Expr, + Validated: activeChecks[i].Validity == sqlbase.ConstraintValidity_Validated, + }) + } + ot.checkConstraints = append(ot.checkConstraints, synthesizedChecks...) + // Add stats last, now that other metadata is initialized. if stats != nil { ot.stats = make([]optTableStat, len(stats)) @@ -781,16 +835,12 @@ func (ot *optTable) Statistic(i int) cat.TableStatistic { // CheckCount is part of the cat.Table interface. func (ot *optTable) CheckCount() int { - return len(ot.desc.ActiveChecks()) + return len(ot.checkConstraints) } // Check is part of the cat.Table interface. func (ot *optTable) Check(i int) cat.CheckConstraint { - check := ot.desc.ActiveChecks()[i] - return cat.CheckConstraint{ - Constraint: check.Expr, - Validated: check.Validity == sqlbase.ConstraintValidity_Validated, - } + return ot.checkConstraints[i] } // FamilyCount is part of the cat.Table interface. diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 0ffb1350e354..49898cb71cb3 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -1769,6 +1769,30 @@ Note If the result has zero or one points, it will be returned as a POINT. If it // // Binary functions // + "st_azimuth": makeBuiltin( + defProps(), + geometryOverload2( + func(ctx *tree.EvalContext, a, b *tree.DGeometry) (tree.Datum, error) { + azimuth, err := geomfn.Azimuth(a.Geometry, b.Geometry) + if err != nil { + return nil, err + } + + if azimuth == nil { + return tree.DNull, nil + } + + return tree.NewDFloat(tree.DFloat(*azimuth)), nil + }, + types.Float, + infoBuilder{ + info: `Returns the azimuth in radians of the segment defined by the given point geometries, or NULL if the two points are coincident. + +The azimuth is angle is referenced from north, and is positive clockwise: North = 0; East = π/2; South = π; West = 3π/2.`, + }, + tree.VolatilityImmutable, + ), + ), "st_distance": makeBuiltin( defProps(), geometryOverload2( diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index b0571b388ed4..b884167af41f 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -3837,6 +3837,21 @@ func MakeDEnumFromLogicalRepresentation(typ *types.T, rep string) (*DEnum, error }, nil } +// MakeAllDEnumsInType generates a slice of all values in an enum. +// TODO (rohany): In the future, take an option of whether to include +// non-writeable enum values or not. +func MakeAllDEnumsInType(typ *types.T) []Datum { + result := make([]Datum, len(typ.TypeMeta.EnumData.LogicalRepresentations)) + for i := 0; i < len(result); i++ { + result[i] = &DEnum{ + EnumTyp: typ, + PhysicalRep: typ.TypeMeta.EnumData.PhysicalRepresentations[i], + LogicalRep: typ.TypeMeta.EnumData.LogicalRepresentations[i], + } + } + return result +} + // Format implements the NodeFormatter interface. func (d *DEnum) Format(ctx *FmtCtx) { if ctx.HasFlags(fmtStaticallyFormatUserDefinedTypes) {