Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
49284: opt: synthesize check constraints on enum columns r=rohany a=rohany

Fixes #49263.

This PR teaches the optimizer how to synthesize check constraints on
columns of an ENUM type, allowing queries like:

```
CREATE TYPE t AS ENUM ('howdy', 'hello');
CREATE TABLE tt (x t, y INT, PRIMARY KEY (x, y));
SELECT x, y FROM tt WHERE y = 2
```

to be planned using constrained spans on the enum values, rather than a
full table scan.

Release note (performance improvement): Allow the optimizer to use enum
information to generate better query plans.

50001: colexec: minor miscellaneous additions r=yuzefovich a=yuzefovich

**colexec: add casts from datum-backed types to bools**

While investigating unrelated test failures, I added this cast, so we
might as well merge it.

Addresses: #48135.

Release note: None

**colexec: add support for Values core with zero rows**

Release note: None

50188: geo/geogfn: implement ST_Azimuth({geometry,geometry}) r=otan a=hueypark

Fixes #48887

Release note (sql change): This PR implement adds the ST_Azimuth({geometry,geometry})

50205: Makefile: ensure redact_safe.md generation is stable r=RichardJCai a=knz

Fixes #50146

On some platforms, the default behavior of `git grep` is to number
the lines (i.e. `-n` is implicitly added).

This patch makes the `-n` explicit and tweaks the sed patterns to
ensure that the behavior is stable.

Release note: None

50239: kv/kvserver: disable the below-raft proto tracking in race builds r=petermattis a=petermattis

The below-raft proto tracking is meant to catch bugs where we
inadvertently starting marshaling a proto below Raft. This tracking is a
source of signficant memory allocations which measurably slow down race
builds. Since we're already doing this tracking in non-race builds,
there is little benefit to also doing it in race builds. Disabling
below-raft proto tracking for race builds reduces the time to run
`testrace` on the `kv/kvserver` package from 605s to 517s on my laptop.

Release note: None

50243: kv/kvserver: fix TestSideloadingSideloadedStorage w/ RocksDB r=jbowens a=jbowens

Fix the TestSideloadingSideloadedStorage test when running with
COCKROACH_STORAGE_ENGINE=rocksdb. The test has always depended on the
exact error message surfaced from os.Remove. In #49717,
diskSideloadStorage was modified to use the engine's RemoveDir operation
rather than interacting with the filesystem directly. Since Pebble uses
os.Remove for its implementation and emulates its error messages in its
MemFS implementation, the exact message comparison continued to succeed.

After #49717, when running with RocksDB, the RocksDB env's error message
was surfaced, with its own context prefixing. This updates the test to
case-insensitively check for 'directory not empty' instead.

Fixes #50226.

Release note: None

Co-authored-by: Rohan Yadav <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Jaewan Park <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Peter Mattis <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
  • Loading branch information
7 people committed Jun 15, 2020
7 parents 1335676 + 415a7b5 + 743af04 + 24585a9 + 8f65201 + c497e76 + 25385d4 commit 5cff000
Show file tree
Hide file tree
Showing 22 changed files with 479 additions and 95 deletions.
10 changes: 5 additions & 5 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
3 changes: 3 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -776,6 +776,9 @@ has no relationship with the commit order of concurrent transactions.</p>
</span></td></tr>
<tr><td><a name="st_astext"></a><code>st_astext(geometry: geometry, maximum_decimal_digits: <a href="int.html">int</a>) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>Returns the WKT representation of a given Geometry. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the <code>.</code>. Use -1 to print as many digits as possible.</p>
</span></td></tr>
<tr><td><a name="st_azimuth"></a><code>st_azimuth(geometry_a: geometry, geometry_b: geometry) &rarr; <a href="float.html">float</a></code></td><td><span class="funcdesc"><p>Returns the azimuth in radians of the segment defined by the given point geometries, or NULL if the two points are coincident.</p>
<p>The azimuth is angle is referenced from north, and is positive clockwise: North = 0; East = π/2; South = π; West = 3π/2.</p>
</span></td></tr>
<tr><td><a name="st_buffer"></a><code>st_buffer(geometry: geometry, distance: <a href="float.html">float</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns a Geometry that represents all points whose distance is less than or equal to the given distance
from the given Geometry.</p>
<p>This function utilizes the GEOS module.</p>
Expand Down
6 changes: 6 additions & 0 deletions pkg/col/coldataext/datum_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)}
Expand Down
58 changes: 58 additions & 0 deletions pkg/geo/geomfn/azimuth.go
Original file line number Diff line number Diff line change
@@ -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
}
76 changes: 76 additions & 0 deletions pkg/geo/geomfn/azimuth_test.go
Original file line number Diff line number Diff line change
@@ -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)
}
})
}
}
10 changes: 10 additions & 0 deletions pkg/kv/kvserver/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand All @@ -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.
Expand Down
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/replica_sideload_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/bool_vec_to_sel.go
Original file line number Diff line number Diff line change
Expand Up @@ -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])
}
Expand Down
93 changes: 64 additions & 29 deletions pkg/sql/colexec/cast_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand All @@ -53,66 +54,100 @@ 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
// Make an input vector of length n.
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)
})
Expand Down
Loading

0 comments on commit 5cff000

Please sign in to comment.